[feature](Load)Suppot MySQL Load Data (#15511)

Main subtask of [DSIP-28](https://cwiki.apache.org/confluence/display/DORIS/DSIP-028%3A+Suppot+MySQL+Load+Data)

## Problem summary
Support mysql load syntax as below: 
```sql
LOAD DATA
    [LOCAL]
    INFILE 'file_name'
    INTO TABLE tbl_name
    [PARTITION (partition_name [, partition_name] ...)]
    [COLUMNS TERMINATED BY 'string']
    [LINES TERMINATED BY 'string']
    [IGNORE number {LINES | ROWS}]
    [(col_name_or_user_var [, col_name_or_user_var] ...)]
    [SET (col_name={expr | DEFAULT} [, col_name={expr | DEFAULT}] ...)]
    [PROPERTIES (key1 = value1 [, key2=value2]) ]
```

For example, 
```sql
            LOAD DATA 
            LOCAL
            INFILE 'local_test.file'
            INTO TABLE db1.table1
            PARTITION (partition_a, partition_b, partition_c, partition_d)
            COLUMNS TERMINATED BY '\t'
            (k1, k2, v2, v10, v11)
            set (c1=k1,c2=k2,c3=v10,c4=v11)
            PROPERTIES ("auth" = "root:", "strict_mode"="true")
```

Note that in this pr the property named `auth` must be set since stream load need auth. I will optimize it later.
This commit is contained in:
huangzhaowei
2023-01-29 14:44:59 +08:00
committed by GitHub
parent c9f66250a8
commit c6bc0a03a4
29 changed files with 1090 additions and 29 deletions

View File

@ -0,0 +1,105 @@
// Licensed to the Apache Software Foundation (ASF) under one
// or more contributor license agreements. See the NOTICE file
// distributed with this work for additional information
// regarding copyright ownership. The ASF licenses this file
// to you under the Apache License, Version 2.0 (the
// "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing,
// software distributed under the License is distributed on an
// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
// KIND, either express or implied. See the License for the
// specific language governing permissions and limitations
// under the License.
package org.apache.doris.common.io;
import java.io.ByteArrayInputStream;
import java.io.IOException;
import java.io.InputStream;
import java.nio.ByteBuffer;
import java.util.concurrent.ArrayBlockingQueue;
import java.util.concurrent.TimeUnit;
public class ByteBufferNetworkInputStream extends InputStream {
private ArrayBlockingQueue<ByteArrayInputStream> queue;
private ByteArrayInputStream currentInputStream;
private volatile boolean finished = false;
private volatile boolean closed = false;
public ByteBufferNetworkInputStream() {
this(32);
}
public ByteBufferNetworkInputStream(int capacity) {
this.queue = new ArrayBlockingQueue<>(capacity);
}
public void fillByteBuffer(ByteBuffer buffer) throws IOException, InterruptedException {
if (closed) {
throw new IOException("Stream is already closed.");
}
ByteArrayInputStream inputStream = new ByteArrayInputStream(buffer.array(), buffer.position(), buffer.limit());
queue.offer(inputStream, 300, TimeUnit.SECONDS);
}
public void markFinished() {
this.finished = true;
}
private ByteArrayInputStream getNextByteArrayStream() throws IOException {
if (currentInputStream == null || currentInputStream.available() == 0) {
// No any byte array stream will come
while (!finished || !queue.isEmpty()) {
try {
currentInputStream = queue.poll(1, TimeUnit.SECONDS);
if (currentInputStream != null) {
return currentInputStream;
}
} catch (InterruptedException e) {
throw new IOException("Failed to get next stream");
}
}
return null;
}
return currentInputStream;
}
@Override
public int read() throws IOException {
ByteArrayInputStream stream = getNextByteArrayStream();
if (stream == null) {
return -1;
}
return stream.read();
}
public int read(byte[] b, int off, int len) throws IOException {
ByteArrayInputStream stream = getNextByteArrayStream();
if (stream == null) {
return -1;
}
return stream.read(b, off, len);
}
public int read(byte[] b) throws IOException {
return read(b, 0, b.length);
}
public void close() throws IOException {
closed = true;
ByteArrayInputStream stream = getNextByteArrayStream();
if (stream == null) {
return;
}
stream.close();
while (!queue.isEmpty()) {
queue.poll().close();
}
}
}

View File

@ -0,0 +1,122 @@
// Licensed to the Apache Software Foundation (ASF) under one
// or more contributor license agreements. See the NOTICE file
// distributed with this work for additional information
// regarding copyright ownership. The ASF licenses this file
// to you under the Apache License, Version 2.0 (the
// "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing,
// software distributed under the License is distributed on an
// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
// KIND, either express or implied. See the License for the
// specific language governing permissions and limitations
// under the License.
package org.apache.doris.common.io;
import org.junit.Assert;
import org.junit.Test;
import java.io.BufferedReader;
import java.io.IOException;
import java.io.InputStreamReader;
import java.nio.ByteBuffer;
public class ByteBufferNetworkInputStreamTest {
@Test
public void testMultiByteBuffer() throws IOException, InterruptedException {
ByteBufferNetworkInputStream inputStream = new ByteBufferNetworkInputStream(2);
inputStream.fillByteBuffer(ByteBuffer.wrap("1\t2\n".getBytes()));
inputStream.fillByteBuffer(ByteBuffer.wrap("2\t3\n".getBytes()));
inputStream.markFinished();
BufferedReader bufferedReader = new BufferedReader(new InputStreamReader(inputStream));
Assert.assertEquals(bufferedReader.readLine(), "1\t2");
Assert.assertEquals(bufferedReader.readLine(), "2\t3");
Assert.assertNull(bufferedReader.readLine());
bufferedReader.close();
}
@Test
public void testMultiThreadByteBuffer() throws IOException, InterruptedException {
int num = 5;
ByteBufferNetworkInputStream inputStream = new ByteBufferNetworkInputStream(2);
Thread thread1 = new Thread(() -> {
try {
for (int i = 0; i < num; i++) {
inputStream.fillByteBuffer(ByteBuffer.wrap(String.format("%d\t%d\n", i, i + 1).getBytes()));
Thread.sleep(500);
}
inputStream.markFinished();
} catch (Exception e) {
e.printStackTrace();
}
});
thread1.start();
Thread thread2 = new Thread(() -> {
try {
BufferedReader bufferedReader = new BufferedReader(new InputStreamReader(inputStream));
int count = 0;
String line = bufferedReader.readLine();
while (line != null) {
Assert.assertEquals(line, String.format("%d\t%d", count, count + 1));
count++;
line = bufferedReader.readLine();
}
Assert.assertEquals(count, num);
} catch (Exception e) {
e.printStackTrace();
}
});
thread2.start();
thread2.join();
Assert.assertFalse(thread1.isAlive());
inputStream.close();
}
@Test
public void testMultiThreadByteBuffer2() throws IOException, InterruptedException {
int num = 5;
ByteBufferNetworkInputStream inputStream = new ByteBufferNetworkInputStream(2);
Thread thread1 = new Thread(() -> {
try {
for (int i = 0; i < num; i++) {
inputStream.fillByteBuffer(ByteBuffer.wrap(String.format("%d\t%d\n", i, i + 1).getBytes()));
}
inputStream.markFinished();
} catch (Exception e) {
e.printStackTrace();
}
});
thread1.start();
Thread thread2 = new Thread(() -> {
try {
BufferedReader bufferedReader = new BufferedReader(new InputStreamReader(inputStream));
int count = 0;
String line = bufferedReader.readLine();
while (line != null) {
Assert.assertEquals(line, String.format("%d\t%d", count, count + 1));
count++;
Thread.sleep(500);
line = bufferedReader.readLine();
}
Assert.assertEquals(count, num);
} catch (Exception e) {
e.printStackTrace();
}
});
thread2.start();
thread2.join();
Assert.assertFalse(thread1.isAlive());
inputStream.close();
}
}