Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Memory reduction. #477

Draft
wants to merge 1 commit into
base: main
Choose a base branch
from
Draft
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -9,6 +9,7 @@
import com.clickhouse.kafka.connect.sink.state.StateProvider;
import com.clickhouse.kafka.connect.sink.state.provider.InMemoryState;
import com.clickhouse.kafka.connect.sink.state.provider.KeeperStateProvider;
import com.clickhouse.kafka.connect.util.Memory;
import com.clickhouse.kafka.connect.util.jmx.ExecutionTimer;
import com.clickhouse.kafka.connect.util.jmx.MBeanServerUtils;
import com.clickhouse.kafka.connect.util.jmx.SinkTaskStatistics;
Expand Down Expand Up @@ -85,19 +86,22 @@ public void put(final Collection<SinkRecord> records) throws IOException, Execut
LOGGER.trace(String.format("Got %d records from put API.", records.size()));
ExecutionTimer processingTime = ExecutionTimer.start();

LOGGER.debug(String.format("Memory: before conversion: %s", Memory.get()));
Map<String, List<Record>> dataRecords = records.stream()
.map(v -> Record.convert(v,
clickHouseSinkConfig.isEnableDbTopicSplit(),
clickHouseSinkConfig.getDbTopicSplitChar(),
clickHouseSinkConfig.getDatabase() ))
.collect(Collectors.groupingBy(Record::getTopicAndPartition));
statistics.recordProcessingTime(processingTime);
LOGGER.debug(String.format("Memory: before processing: %s", Memory.get()));
// TODO - Multi process???
for (String topicAndPartition : dataRecords.keySet()) {
// Running on etch topic & partition
List<Record> rec = dataRecords.get(topicAndPartition);
processing.doLogic(rec);
}
LOGGER.debug(String.format("Memory: after processing: %s", Memory.get()));
statistics.taskProcessingTime(taskTime);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -40,6 +40,9 @@ public Record(SchemaType schemaType, OffsetContainer recordOffsetContainer, List
this.database = database;
}

public void setJsonMap(Map<String, Data> jsonMap) {
this.jsonMap = jsonMap;
}
public String getTopicAndPartition() {
return recordOffsetContainer.getTopicAndPartitionKey();
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,7 +18,7 @@ public Record doConvert(SinkRecord sinkRecord, String topic,String configuration
int partition = sinkRecord.kafkaPartition().intValue();
long offset = sinkRecord.kafkaOffset();
Struct struct = (Struct) sinkRecord.value();
Map<String, Data> data = StructToJsonMap.toJsonMap((Struct) sinkRecord.value());
Map<String, Data> data = null; //StructToJsonMap.toJsonMap((Struct) sinkRecord.value());
return new Record(SchemaType.SCHEMA, new OffsetContainer(topic, partition, offset), struct.schema().fields(), data, database, sinkRecord);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -699,6 +699,8 @@ protected void doInsertRawBinaryV2(List<Record> records, Table table, QueryIdent
Record first = records.get(0);
String database = first.getDatabase();

Map<String, Data> dataRecord = StructToJsonMap.toJsonMap((Struct) first.getSinkRecord().value());
first.setJsonMap(dataRecord);
if (!csc.isBypassSchemaValidation() && !validateDataSchema(table, first, false))
throw new RuntimeException("Data schema validation failed.");
// Let's test first record
Expand All @@ -722,12 +724,15 @@ protected void doInsertRawBinaryV2(List<Record> records, Table table, QueryIdent
ByteArrayOutputStream stream = new ByteArrayOutputStream();
for (Record record : records) {
if (record.getSinkRecord().value() != null) {
Map<String, Data> dataRecordTmp = StructToJsonMap.toJsonMap((Struct) record.getSinkRecord().value());
record.setJsonMap(dataRecordTmp);
for (Column col : table.getRootColumnsList()) {
LOGGER.debug("Writing column: {}", col.getName());
long beforePushStream = System.currentTimeMillis();
doWriteCol(record, col, stream, supportDefaults);
pushStreamTime += System.currentTimeMillis() - beforePushStream;
}
record.setJsonMap(null);
}
}

Expand All @@ -753,6 +758,9 @@ protected void doInsertRawBinaryV1(List<Record> records, Table table, QueryIdent
Record first = records.get(0);
String database = first.getDatabase();

Map<String, Data> dataRecord = StructToJsonMap.toJsonMap((Struct) first.getSinkRecord().value());
first.setJsonMap(dataRecord);

if (!csc.isBypassSchemaValidation() && !validateDataSchema(table, first, false))
throw new RuntimeException("Data schema validation failed.");
// Let's test first record
Expand All @@ -776,11 +784,15 @@ protected void doInsertRawBinaryV1(List<Record> records, Table table, QueryIdent
// write bytes into the piped stream
for (Record record : records) {
if (record.getSinkRecord().value() != null) {
Map<String, Data> dataRecordTmp = StructToJsonMap.toJsonMap((Struct) record.getSinkRecord().value());
record.setJsonMap(dataRecordTmp);
for (Column col : table.getRootColumnsList()) {
long beforePushStream = System.currentTimeMillis();
doWriteCol(record, col, stream, supportDefaults);
pushStreamTime += System.currentTimeMillis() - beforePushStream;
}
record.setJsonMap(null);

}
}
// We need to close the stream before getting a response
Expand Down
16 changes: 16 additions & 0 deletions src/main/java/com/clickhouse/kafka/connect/util/Memory.java
Original file line number Diff line number Diff line change
@@ -0,0 +1,16 @@
package com.clickhouse.kafka.connect.util;

public class Memory {
long free = Runtime.getRuntime().freeMemory();
long total = Runtime.getRuntime().totalMemory();
long max = Runtime.getRuntime().maxMemory();
long used = total - free;

public static Memory get() {
return new Memory();
}

public String toString() {
return String.format("Memory: free:[%s] total:[%s] max:[%s] used:[%s]", free, total, max, used);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -5,6 +5,7 @@
import com.clickhouse.kafka.connect.sink.helper.SchemaTestData;
import com.clickhouse.kafka.connect.sink.junit.extension.FromVersionConditionExtension;
import com.clickhouse.kafka.connect.sink.junit.extension.SinceClickHouseVersion;
import com.clickhouse.kafka.connect.util.Memory;
import com.clickhouse.kafka.connect.util.Utils;
import org.apache.kafka.connect.errors.DataException;
import org.apache.kafka.connect.sink.SinkRecord;
Expand All @@ -31,6 +32,7 @@ public class ClickHouseSinkTaskWithSchemaTest extends ClickHouseBase {
private static final Logger LOGGER = LoggerFactory.getLogger(ClickHouseSinkTaskWithSchemaTest.class);
@Test
public void arrayTypesTest() {
LOGGER.debug(String.format("Memory: before test start: %s", Memory.get()));
Map<String, String> props = createProps();
ClickHouseHelperClient chc = createClient(props);

Expand All @@ -42,12 +44,12 @@ public void arrayTypesTest() {
"`arr_map` Array(Map(String, String)) ) Engine = MergeTree ORDER BY off16");
// https://github.com/apache/kafka/blob/trunk/connect/api/src/test/java/org/apache/kafka/connect/data/StructTest.java#L95-L98
Collection<SinkRecord> sr = SchemaTestData.createArrayType(topic, 1);
LOGGER.debug(String.format("Memory: before test start: %s", Memory.get()));

ClickHouseSinkTask chst = new ClickHouseSinkTask();
chst.start(props);
chst.put(sr);
chst.stop();

assertEquals(sr.size(), ClickHouseTestHelpers.countRows(chc, topic));
assertTrue(ClickHouseTestHelpers.validateRows(chc, topic, sr));
}
Expand Down
Loading