Skip to content
Closed
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 @@ -19,9 +19,6 @@
package org.apache.paimon.client;

import java.io.Closeable;
import java.util.ArrayList;
import java.util.List;
import java.util.concurrent.LinkedBlockingDeque;
import java.util.concurrent.TimeUnit;
import java.util.function.Supplier;

Expand All @@ -42,32 +39,29 @@ interface ExecuteAction<C, E extends Exception> {
void execute(ExecuteAction<C, E> action) throws E, InterruptedException;

/** Default implementation for {@link ClientPool}. */
abstract class ClientPoolImpl<C, E extends Exception> implements Closeable, ClientPool<C, E> {
abstract class ClientPoolImpl<C, P, E extends Exception>
implements Closeable, ClientPool<C, E> {
protected ClientPoolImpl(Supplier<P> supplier) {
initPool(supplier);
}

private volatile LinkedBlockingDeque<C> clients;
protected abstract void initPool(Supplier<P> supplier);

protected ClientPoolImpl(int poolSize, Supplier<C> supplier) {
this.clients = new LinkedBlockingDeque<>();
for (int i = 0; i < poolSize; i++) {
this.clients.add(supplier.get());
}
}
protected abstract C getClient(long timeout, TimeUnit unit) throws E, InterruptedException;

protected abstract void recycleClient(C client) throws E, InterruptedException;

@Override
public <R> R run(Action<R, C, E> action) throws E, InterruptedException {
while (true) {
LinkedBlockingDeque<C> clients = this.clients;
if (clients == null) {
throw new IllegalStateException("Cannot get a client from a closed pool");
}
C client = clients.pollFirst(10, TimeUnit.SECONDS);
C client = getClient(10, TimeUnit.SECONDS);
if (client == null) {
continue;
}
try {
return action.run(client);
} finally {
clients.addFirst(client);
recycleClient(client);
}
}
}
Expand All @@ -82,17 +76,11 @@ public void execute(ExecuteAction<C, E> action) throws E, InterruptedException {
});
}

protected abstract void close(C client);
protected abstract void closePool();

@Override
public void close() {
LinkedBlockingDeque<C> clients = this.clients;
this.clients = null;
if (clients != null) {
List<C> drain = new ArrayList<>();
clients.drainTo(drain);
drain.forEach(this::close);
}
closePool();
}
}
}
12 changes: 12 additions & 0 deletions paimon-core/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -246,6 +246,18 @@ under the License.
<version>42.7.3</version>
</dependency>

<dependency>
<groupId>com.alibaba</groupId>
<artifactId>druid</artifactId>
<version>1.2.24</version>
</dependency>

<dependency>
<groupId>mysql</groupId>
<artifactId>mysql-connector-java</artifactId>
<version>8.0.28</version>
<scope>test</scope>
</dependency>

</dependencies>

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,24 +20,30 @@

import org.apache.paimon.client.ClientPool;

import com.alibaba.druid.pool.DruidDataSource;
import com.alibaba.druid.pool.DruidDataSourceFactory;

import java.sql.Connection;
import java.sql.DriverManager;
import java.sql.SQLException;
import java.util.Map;
import java.util.Properties;
import java.util.concurrent.TimeUnit;
import java.util.function.Supplier;
import java.util.regex.Matcher;
import java.util.regex.Pattern;

/** Client pool for jdbc. */
public class JdbcClientPool extends ClientPool.ClientPoolImpl<Connection, SQLException> {
public class JdbcClientPool
extends ClientPool.ClientPoolImpl<Connection, DruidDataSource, SQLException> {

private static final Pattern PROTOCOL_PATTERN = Pattern.compile("jdbc:([^:]+):(.*)");

private final String protocol;

private volatile DruidDataSource dataSource;

public JdbcClientPool(int poolSize, String dbUrl, Map<String, String> props) {
super(poolSize, clientSupplier(dbUrl, props));
super(initPoolSupplier(poolSize, dbUrl, props));
Matcher matcher = PROTOCOL_PATTERN.matcher(dbUrl);
if (matcher.matches()) {
this.protocol = matcher.group(1);
Expand All @@ -46,28 +52,50 @@ public JdbcClientPool(int poolSize, String dbUrl, Map<String, String> props) {
}
}

private static Supplier<Connection> clientSupplier(String dbUrl, Map<String, String> props) {
public String getProtocol() {
return protocol;
}

private static Supplier<DruidDataSource> initPoolSupplier(
int poolSize, String dbUrl, Map<String, String> props) {
return () -> {
try {
Properties dbProps =
JdbcUtils.extractJdbcConfiguration(props, JdbcCatalog.PROPERTY_PREFIX);
return DriverManager.getConnection(dbUrl, dbProps);
} catch (SQLException e) {
throw new RuntimeException(String.format("Failed to connect: %s", dbUrl), e);
dbProps.setProperty(DruidDataSourceFactory.PROP_URL, dbUrl);
dbProps.setProperty(
DruidDataSourceFactory.PROP_MAXACTIVE, String.valueOf(poolSize));
return (DruidDataSource) DruidDataSourceFactory.createDataSource(dbProps);
} catch (Exception e) {
throw new RuntimeException("Failed to create datasource", e);
}
};
}

public String getProtocol() {
return protocol;
@Override
protected void initPool(Supplier<DruidDataSource> supplier) {
dataSource = supplier.get();
}

@Override
protected Connection getClient(long timeout, TimeUnit unit) throws SQLException {
if (this.dataSource == null) {
throw new IllegalStateException("Cannot get a client from a closed pool");
}
return dataSource.getConnection(unit.toMillis(timeout));
}

@Override
protected void recycleClient(Connection client) throws SQLException {
client.close();
}

@Override
protected void close(Connection client) {
try {
client.close();
} catch (SQLException e) {
throw new RuntimeException("Failed to close connection", e);
protected void closePool() {
DruidDataSource dataSource = this.dataSource;
this.dataSource = null;
if (dataSource != null) {
dataSource.close();
}
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,56 @@
/*
* 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.paimon.jdbc;

import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test;
import org.testcontainers.shaded.com.google.common.collect.Maps;

import java.sql.Connection;
import java.sql.SQLException;
import java.util.HashMap;
import java.util.concurrent.TimeUnit;

import static org.assertj.core.api.Assertions.assertThat;

/** Tests for {@link JdbcClientPool}. */
public class JdbcClientPoolTest {
public JdbcClientPool connections;

@BeforeEach
public void setUp() {
connections = initConnections(Maps.newHashMap());
}

private JdbcClientPool initConnections(HashMap<String, String> props) {
props.put("jdbc.autoReconnect", "true");
props.put("jdbc.testOnBorrow", "true");
props.put("jdbc.validationQuery", "SELECT 1");

String testUrl = "jdbc:mysql://127.0.0.1:33306/mysql?user=root&password=123456";
return new JdbcClientPool(10, testUrl, props);
}

@Test
public void testGetClient() throws SQLException, InterruptedException {
Connection client = connections.getClient(10, TimeUnit.SECONDS);
String lockId = "jdbc.testDb.testTable";
assertThat(JdbcUtils.acquire(connections, lockId, 1000)).isTrue();
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -26,28 +26,68 @@
import org.apache.hadoop.hive.metastore.IMetaStoreClient;
import org.apache.thrift.TException;

import java.util.ArrayList;
import java.util.List;
import java.util.concurrent.LinkedBlockingDeque;
import java.util.concurrent.TimeUnit;
import java.util.function.Supplier;

/**
* Pool of Hive Metastore clients.
*
* <p>Mostly copied from iceberg.
*/
public class HiveClientPool extends ClientPool.ClientPoolImpl<IMetaStoreClient, TException> {
public class HiveClientPool
extends ClientPool.ClientPoolImpl<
IMetaStoreClient, LinkedBlockingDeque<IMetaStoreClient>, TException> {

private volatile LinkedBlockingDeque<IMetaStoreClient> clients;

public HiveClientPool(int poolSize, Configuration conf, String clientClassName) {
super(poolSize, clientSupplier(conf, clientClassName));
super(initPoolSupplier(poolSize, conf, clientClassName));
}

private static Supplier<LinkedBlockingDeque<IMetaStoreClient>> initPoolSupplier(
int poolSize, Configuration conf, String clientClassName) {
return () -> {
LinkedBlockingDeque<IMetaStoreClient> clients = new LinkedBlockingDeque<>();
HiveConf hiveConf = new HiveConf(conf, HiveClientPool.class);
hiveConf.addResource(conf);
for (int i = 0; i < poolSize; i++) {
clients.add(
new RetryingMetaStoreClientFactory()
.createClient(hiveConf, clientClassName));
}
return clients;
};
}

@Override
protected void initPool(Supplier<LinkedBlockingDeque<IMetaStoreClient>> supplier) {
this.clients = supplier.get();
}

private static Supplier<IMetaStoreClient> clientSupplier(
Configuration conf, String clientClassName) {
HiveConf hiveConf = new HiveConf(conf, HiveClientPool.class);
hiveConf.addResource(conf);
return () -> new RetryingMetaStoreClientFactory().createClient(hiveConf, clientClassName);
@Override
protected IMetaStoreClient getClient(long timeout, TimeUnit unit) throws InterruptedException {
if (this.clients == null) {
throw new IllegalStateException("Cannot get a client from a closed pool");
}
return this.clients.pollFirst(10, TimeUnit.SECONDS);
}

@Override
protected void recycleClient(IMetaStoreClient client) {
this.clients.addFirst(client);
}

@Override
protected void close(IMetaStoreClient client) {
client.close();
protected void closePool() {
LinkedBlockingDeque<IMetaStoreClient> clients = this.clients;
this.clients = null;
if (clients != null) {
List<IMetaStoreClient> drain = new ArrayList<>();
clients.drainTo(drain);
drain.forEach(IMetaStoreClient::close);
}
}
}