Skip to content

Commit

Permalink
Merge remote-tracking branch 'origin/main' into bitmap-serialize
Browse files Browse the repository at this point in the history
  • Loading branch information
javiercj93 committed Sep 12, 2024
2 parents 13e8b89 + f9b0f7a commit 2539301
Show file tree
Hide file tree
Showing 35 changed files with 2,557 additions and 316 deletions.
2 changes: 1 addition & 1 deletion .github/workflows/analysis.yml
Original file line number Diff line number Diff line change
Expand Up @@ -95,6 +95,6 @@ jobs:
GITHUB_TOKEN: ${{ secrets.GITHUB_TOKEN }} # Needed to get PR information, if any
SONAR_TOKEN: ${{ secrets.SONAR_TOKEN }}
run: |
mvn --batch-mode -DclickhouseVersion=$PREFERRED_LTS_VERSION \
mvn --batch-mode -DclickhouseVersion=$PREFERRED_LTS_VERSION -Dclient.tests.useNewImplementation=true \
-Panalysis verify org.sonarsource.scanner.maven:sonar-maven-plugin:sonar
continue-on-error: true
8 changes: 8 additions & 0 deletions client-v2/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -85,6 +85,14 @@
<scope>provided</scope>
</dependency>

<!-- https://mvnrepository.com/artifact/org.ow2.asm/asm -->
<dependency>
<groupId>org.ow2.asm</groupId>
<artifactId>asm</artifactId>
<version>9.7</version>
</dependency>


<!-- Test dependencies -->
<dependency>
<groupId>${project.parent.groupId}</groupId>
Expand Down
254 changes: 191 additions & 63 deletions client-v2/src/main/java/com/clickhouse/client/api/Client.java

Large diffs are not rendered by default.

Original file line number Diff line number Diff line change
Expand Up @@ -19,7 +19,7 @@
import java.util.Map;
import java.util.UUID;

public interface ClickHouseBinaryFormatReader {
public interface ClickHouseBinaryFormatReader extends AutoCloseable {

/**
* Reads a single value from the stream.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -23,12 +23,9 @@ public class NativeFormatReader extends AbstractBinaryFormatReader {

private int blockRowIndex;

public NativeFormatReader(InputStream inputStream) {
this(inputStream, null);
}

public NativeFormatReader(InputStream inputStream, QuerySettings settings) {
super(inputStream, settings, null);
public NativeFormatReader(InputStream inputStream, QuerySettings settings,
BinaryStreamReader.ByteBufferAllocator byteBufferAllocator) {
super(inputStream, settings, null, byteBufferAllocator);
readNextRecord();
}

Expand Down
Original file line number Diff line number Diff line change
@@ -1,6 +1,7 @@
package com.clickhouse.client.api.data_formats;

import com.clickhouse.client.api.data_formats.internal.AbstractBinaryFormatReader;
import com.clickhouse.client.api.data_formats.internal.BinaryStreamReader;
import com.clickhouse.client.api.metadata.TableSchema;
import com.clickhouse.client.api.query.QuerySettings;
import com.clickhouse.data.ClickHouseColumn;
Expand All @@ -12,12 +13,9 @@

public class RowBinaryFormatReader extends AbstractBinaryFormatReader {

public RowBinaryFormatReader(InputStream inputStream, TableSchema schema) {
this(inputStream, null, schema);
}

public RowBinaryFormatReader(InputStream inputStream, QuerySettings querySettings, TableSchema schema) {
super(inputStream, querySettings, schema);
public RowBinaryFormatReader(InputStream inputStream, QuerySettings querySettings, TableSchema schema,
BinaryStreamReader.ByteBufferAllocator byteBufferAllocator) {
super(inputStream, querySettings, schema, byteBufferAllocator);
readNextRecord();
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -5,7 +5,6 @@
import com.clickhouse.client.api.data_formats.internal.BinaryStreamReader;
import com.clickhouse.client.api.metadata.TableSchema;
import com.clickhouse.client.api.query.QuerySettings;
import com.clickhouse.data.ClickHouseColumn;

import java.io.EOFException;
import java.io.IOException;
Expand All @@ -17,8 +16,9 @@

public class RowBinaryWithNamesAndTypesFormatReader extends AbstractBinaryFormatReader implements Iterator<Map<String, Object>> {

public RowBinaryWithNamesAndTypesFormatReader(InputStream inputStream, QuerySettings querySettings) {
super(inputStream, querySettings, null);
public RowBinaryWithNamesAndTypesFormatReader(InputStream inputStream, QuerySettings querySettings,
BinaryStreamReader.ByteBufferAllocator byteBufferAllocator) {
super(inputStream, querySettings, null, byteBufferAllocator);
readSchema();
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -4,27 +4,21 @@
import com.clickhouse.client.api.data_formats.internal.BinaryStreamReader;
import com.clickhouse.client.api.metadata.TableSchema;
import com.clickhouse.client.api.query.QuerySettings;
import com.clickhouse.data.ClickHouseColumn;

import java.io.EOFException;
import java.io.IOException;
import java.io.InputStream;
import java.util.ArrayList;
import java.util.Collections;
import java.util.List;
import java.util.Map;

public class RowBinaryWithNamesFormatReader extends AbstractBinaryFormatReader {

private List<String> columns = null;

public RowBinaryWithNamesFormatReader(InputStream inputStream, TableSchema schema) {
this(inputStream, null, schema);
readNextRecord();
}

public RowBinaryWithNamesFormatReader(InputStream inputStream, QuerySettings querySettings, TableSchema schema) {
super(inputStream, querySettings, schema);
public RowBinaryWithNamesFormatReader(InputStream inputStream, QuerySettings querySettings, TableSchema schema,
BinaryStreamReader.ByteBufferAllocator byteBufferAllocator) {
super(inputStream, querySettings, schema, byteBufferAllocator);
int nCol = 0;
try {
nCol = BinaryStreamReader.readVarInt(input);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -2,13 +2,12 @@

import com.clickhouse.client.api.ClientException;
import com.clickhouse.client.api.data_formats.ClickHouseBinaryFormatReader;
import com.clickhouse.client.api.internal.MapUtils;
import com.clickhouse.client.api.metadata.TableSchema;
import com.clickhouse.client.api.query.NullValueException;
import com.clickhouse.client.api.query.POJOSetter;
import com.clickhouse.client.api.query.QuerySettings;
import com.clickhouse.client.config.ClickHouseClientOption;
import com.clickhouse.data.ClickHouseColumn;
import com.clickhouse.data.value.ClickHouseArrayValue;
import com.clickhouse.data.value.ClickHouseGeoMultiPolygonValue;
import com.clickhouse.data.value.ClickHouseGeoPointValue;
import com.clickhouse.data.value.ClickHouseGeoPolygonValue;
Expand All @@ -30,17 +29,14 @@
import java.time.ZoneOffset;
import java.time.ZonedDateTime;
import java.time.temporal.ChronoUnit;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.NoSuchElementException;
import java.util.TimeZone;
import java.util.UUID;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.function.Function;

public abstract class AbstractBinaryFormatReader implements ClickHouseBinaryFormatReader {

Expand All @@ -58,7 +54,8 @@ public abstract class AbstractBinaryFormatReader implements ClickHouseBinaryForm

private volatile boolean hasNext = true;

protected AbstractBinaryFormatReader(InputStream inputStream, QuerySettings querySettings, TableSchema schema) {
protected AbstractBinaryFormatReader(InputStream inputStream, QuerySettings querySettings, TableSchema schema,
BinaryStreamReader.ByteBufferAllocator byteBufferAllocator) {
this.input = inputStream;
this.settings = querySettings == null ? Collections.emptyMap() : new HashMap<>(querySettings.getAllSettings());
boolean useServerTimeZone = (boolean) this.settings.get(ClickHouseClientOption.USE_SERVER_TIME_ZONE.getKey());
Expand All @@ -67,7 +64,7 @@ protected AbstractBinaryFormatReader(InputStream inputStream, QuerySettings quer
if (timeZone == null) {
throw new ClientException("Time zone is not set. (useServerTimezone:" + useServerTimeZone + ")");
}
this.binaryStreamReader = new BinaryStreamReader(inputStream, timeZone, LOG);
this.binaryStreamReader = new BinaryStreamReader(inputStream, timeZone, LOG, byteBufferAllocator);
setSchema(schema);
}

Expand All @@ -76,6 +73,32 @@ protected AbstractBinaryFormatReader(InputStream inputStream, QuerySettings quer

protected AtomicBoolean nextRecordEmpty = new AtomicBoolean(true);

public boolean readToPOJO(Map<String, POJOSetter> deserializers, Object obj ) throws IOException {
boolean firstColumn = true;

for (ClickHouseColumn column : columns) {
try {
Object val = binaryStreamReader.readValue(column);
if (val != null) {
POJOSetter deserializer = deserializers.get(column.getColumnName());
if (deserializer != null) {
deserializer.setValue(obj, val);
}
}
firstColumn = false;
} catch (EOFException e) {
if (firstColumn) {
endReached();
return false;
}
throw e;
} catch (Exception e) {
throw new ClientException("Failed to put value of '" + column.getColumnName() + "' into POJO", e);
}
}
return true;
}

/**
* It is still internal method and should be used with care.
* Usually this method is called to read next record into internal object and affects hasNext() method.
Expand Down Expand Up @@ -133,12 +156,12 @@ protected void readNextRecord() {
try {
nextRecordEmpty.set(true);
if (!readRecord(nextRecord)) {
hasNext = false;
endReached();
} else {
nextRecordEmpty.compareAndSet(true, false);
}
} catch (IOException e) {
hasNext = false;
endReached();
throw new ClientException("Failed to read next row", e);
}
}
Expand All @@ -165,7 +188,7 @@ public Map<String, Object> next() {
return null;
}
} catch (IOException e) {
hasNext = false;
endReached();
throw new ClientException("Failed to read row", e);
}
}
Expand Down Expand Up @@ -621,4 +644,9 @@ public LocalDateTime getLocalDateTime(int index) {
}
return (LocalDateTime) value;
}

@Override
public void close() throws Exception {
input.close();
}
}
Loading

0 comments on commit 2539301

Please sign in to comment.