Skip to content

Commit

Permalink
add jdbc catalog
Browse files Browse the repository at this point in the history
  • Loading branch information
sunxiaojian committed Feb 7, 2024
1 parent ca14b89 commit 6a2ab2c
Show file tree
Hide file tree
Showing 14 changed files with 1,635 additions and 29 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -39,7 +39,8 @@ public class CatalogOptions {
ConfigOptions.key("metastore")
.stringType()
.defaultValue("filesystem")
.withDescription("Metastore of paimon catalog, supports filesystem and hive.");
.withDescription(
"Metastore of paimon catalog, supports filesystem、hive and jdbc.");

public static final ConfigOption<String> URI =
ConfigOptions.key("uri")
Expand Down Expand Up @@ -78,6 +79,13 @@ public class CatalogOptions {
.withDescription(
"Allow to fallback to hadoop File IO when no file io found for the scheme.");

public static final ConfigOption<Integer> CLIENT_POOL_SIZE =
key("client-pool-size")
.intType()
.defaultValue(2)
.withDescription(
"Allow to fallback to hadoop File IO when no file io found for the scheme.");

public static final ConfigOption<String> LINEAGE_META =
key("lineage-meta")
.stringType()
Expand Down
14 changes: 14 additions & 0 deletions paimon-core/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -183,6 +183,20 @@ under the License.
<version>3.6.1</version>
</dependency>

<dependency>
<groupId>org.xerial</groupId>
<artifactId>sqlite-jdbc</artifactId>
<version>3.44.0.0</version>
<scope>test</scope>
</dependency>

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

</dependencies>

<build>
Expand Down
29 changes: 29 additions & 0 deletions paimon-core/src/main/java/org/apache/paimon/ClientPool.java
Original file line number Diff line number Diff line change
@@ -0,0 +1,29 @@
/*
* 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;

public interface ClientPool<C, E extends Exception> {
interface Action<R, C, E extends Exception> {
R run(C client) throws E;
}

<R> R run(Action<R, C, E> action) throws E, InterruptedException;

<R> R run(Action<R, C, E> action, boolean retry) throws E, InterruptedException;
}
156 changes: 156 additions & 0 deletions paimon-core/src/main/java/org/apache/paimon/ClientPoolImpl.java
Original file line number Diff line number Diff line change
@@ -0,0 +1,156 @@
/*
* 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;

import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

import java.io.Closeable;
import java.util.ArrayDeque;
import java.util.Deque;

import static org.apache.paimon.utils.Preconditions.checkState;

public abstract class ClientPoolImpl<C, E extends Exception>
implements Closeable, ClientPool<C, E> {
private static final Logger LOG = LoggerFactory.getLogger(ClientPoolImpl.class);

private final int poolSize;
private final Deque<C> clients;
private final Class<? extends E> reconnectExc;
private final Object signal = new Object();
private final boolean retryByDefault;
private volatile int currentSize;
private boolean closed;

public ClientPoolImpl(int poolSize, Class<? extends E> reconnectExc, boolean retryByDefault) {
this.poolSize = poolSize;
this.reconnectExc = reconnectExc;
this.clients = new ArrayDeque<>(poolSize);
this.currentSize = 0;
this.closed = false;
this.retryByDefault = retryByDefault;
}

@Override
public <R> R run(Action<R, C, E> action) throws E, InterruptedException {
return run(action, retryByDefault);
}

@Override
public <R> R run(Action<R, C, E> action, boolean retry) throws E, InterruptedException {
C client = get();
try {
return action.run(client);

} catch (Exception exc) {
if (retry && isConnectionException(exc)) {
try {
client = reconnect(client);
} catch (Exception ignored) {
// if reconnection throws any exception, rethrow the original failure
throw reconnectExc.cast(exc);
}

return action.run(client);
}

throw exc;

} finally {
release(client);
}
}

protected abstract C newClient();

protected abstract C reconnect(C client);

protected boolean isConnectionException(Exception exc) {
return reconnectExc.isInstance(exc);
}

protected abstract void close(C client);

@Override
public void close() {
this.closed = true;
try {
while (currentSize > 0) {
if (!clients.isEmpty()) {
synchronized (this) {
if (!clients.isEmpty()) {
C client = clients.removeFirst();
close(client);
currentSize -= 1;
}
}
}
if (clients.isEmpty() && currentSize > 0) {
// wake every second in case this missed the signal
synchronized (signal) {
signal.wait(1000);
}
}
}

} catch (InterruptedException e) {
Thread.currentThread().interrupt();
LOG.warn("Interrupted while shutting down pool. Some clients may not be closed.", e);
}
}

private C get() throws InterruptedException {
checkState(!closed, "Cannot get a client from a closed pool");
while (true) {
if (!clients.isEmpty() || currentSize < poolSize) {
synchronized (this) {
if (!clients.isEmpty()) {
return clients.removeFirst();
} else if (currentSize < poolSize) {
C client = newClient();
currentSize += 1;
return client;
}
}
}
synchronized (signal) {
// wake every second in case this missed the signal
signal.wait(1000);
}
}
}

private void release(C client) {
synchronized (this) {
clients.addFirst(client);
}
synchronized (signal) {
signal.notify();
}
}

public int poolSize() {
return poolSize;
}

public boolean isClosed() {
return closed;
}
}
Loading

0 comments on commit 6a2ab2c

Please sign in to comment.