Skip to content

Commit

Permalink
[service] Introduce network server and client (apache#2568)
Browse files Browse the repository at this point in the history
  • Loading branch information
JingsongLi authored Dec 27, 2023
1 parent 0b2cc15 commit 9fc5c15
Show file tree
Hide file tree
Showing 39 changed files with 5,030 additions and 0 deletions.
2 changes: 2 additions & 0 deletions LICENSE
Original file line number Diff line number Diff line change
Expand Up @@ -233,6 +233,8 @@ paimon-common/src/main/java/org/apache/paimon/data/InternalRow.java
paimon-common/src/main/java/org/apache/paimon/memory/MemorySegment.java
paimon-common/src/main/java/org/apache/paimon/types/DataType.java
paimon-common/src/main/java/org/apache/paimon/options/ConfigOption.java
paimon-service/paimon-service-client/src/main/java/org/apache/paimon/service/network/NetworkClient.java
paimon-service/paimon-service-client/src/main/java/org/apache/paimon/service/network/NetworkServer.java
from http://flink.apache.org/ version 1.17.0

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

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

import java.util.concurrent.CompletableFuture;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.TimeUnit;

/** Utilities for {@link java.util.concurrent.Executor Executors}. */
public class ExecutorUtils {

private static final Logger LOG = LoggerFactory.getLogger(ExecutorUtils.class);

/**
* Gracefully shutdown the given {@link ExecutorService}. The call waits the given timeout that
* all ExecutorServices terminate. If the ExecutorServices do not terminate in this time, they
* will be shut down hard.
*
* @param timeout to wait for the termination of all ExecutorServices
* @param unit of the timeout
* @param executorServices to shut down
*/
public static void gracefulShutdown(
long timeout, TimeUnit unit, ExecutorService... executorServices) {
for (ExecutorService executorService : executorServices) {
executorService.shutdown();
}

boolean wasInterrupted = false;
final long endTime = unit.toMillis(timeout) + System.currentTimeMillis();
long timeLeft = unit.toMillis(timeout);
boolean hasTimeLeft = timeLeft > 0L;

for (ExecutorService executorService : executorServices) {
if (wasInterrupted || !hasTimeLeft) {
executorService.shutdownNow();
} else {
try {
if (!executorService.awaitTermination(timeLeft, TimeUnit.MILLISECONDS)) {
LOG.warn(
"ExecutorService did not terminate in time. Shutting it down now.");
executorService.shutdownNow();
}
} catch (InterruptedException e) {
LOG.warn(
"Interrupted while shutting down executor services. Shutting all "
+ "remaining ExecutorServices down now.",
e);
executorService.shutdownNow();

wasInterrupted = true;

Thread.currentThread().interrupt();
}

timeLeft = endTime - System.currentTimeMillis();
hasTimeLeft = timeLeft > 0L;
}
}
}

/**
* Shuts the given {@link ExecutorService} down in a non-blocking fashion. The shut down will be
* executed by a thread from the common fork-join pool.
*
* <p>The executor services will be shut down gracefully for the given timeout period.
* Afterwards {@link ExecutorService#shutdownNow()} will be called.
*
* @param timeout before {@link ExecutorService#shutdownNow()} is called
* @param unit time unit of the timeout
* @param executorServices to shut down
* @return Future which is completed once the {@link ExecutorService} are shut down
*/
public static CompletableFuture<Void> nonBlockingShutdown(
long timeout, TimeUnit unit, ExecutorService... executorServices) {
return CompletableFuture.supplyAsync(
() -> {
gracefulShutdown(timeout, unit, executorServices);
return null;
});
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,75 @@
/*
* 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.utils;

import javax.annotation.Nullable;

import java.util.concurrent.CompletableFuture;
import java.util.function.BiConsumer;

/** A collection of utilities that expand the usage of {@link CompletableFuture}. */
public class FutureUtils {

/**
* Returns an exceptionally completed {@link CompletableFuture}.
*
* @param cause to complete the future with
* @param <T> type of the future
* @return An exceptionally completed CompletableFuture
*/
public static <T> CompletableFuture<T> completedExceptionally(Throwable cause) {
CompletableFuture<T> result = new CompletableFuture<>();
result.completeExceptionally(cause);

return result;
}

/**
* Forwards the value from the source future to the target future.
*
* @param source future to forward the value from
* @param target future to forward the value to
* @param <T> type of the value
*/
public static <T> void forward(CompletableFuture<T> source, CompletableFuture<T> target) {
source.whenComplete(forwardTo(target));
}

private static <T> BiConsumer<T, Throwable> forwardTo(CompletableFuture<T> target) {
return (value, throwable) -> doForward(value, throwable, target);
}

/**
* Completes the given future with either the given value or throwable, depending on which
* parameter is not null.
*
* @param value value with which the future should be completed
* @param throwable throwable with which the future should be completed exceptionally
* @param target future to complete
* @param <T> completed future
*/
public static <T> void doForward(
@Nullable T value, @Nullable Throwable throwable, CompletableFuture<T> target) {
if (throwable != null) {
target.completeExceptionally(throwable);
} else {
target.complete(value);
}
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,27 @@
/*
* 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.lookup;

import org.apache.paimon.data.BinaryRow;

/** An interface to provide lookup function. */
public interface QueryLookup {

BinaryRow[] lookup(BinaryRow partition, int bucket, BinaryRow[] keys);
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,41 @@
/*
* 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.lookup;

import java.net.InetSocketAddress;

/**
* An interface for the Server running on each Node in the cluster. This server is responsible for
* serving requests coming from the client.
*/
public interface QueryServer {

/**
* Returns the {@link InetSocketAddress address} the server is listening to.
*
* @return Server address.
*/
InetSocketAddress getServerAddress();

/** Starts the server. */
void start() throws Throwable;

/** Shuts down the server and all related thread pools. */
void shutdown();
}
81 changes: 81 additions & 0 deletions paimon-service/paimon-service-client/pom.xml
Original file line number Diff line number Diff line change
@@ -0,0 +1,81 @@
<?xml version="1.0" encoding="UTF-8"?>
<!--
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.
-->
<project xmlns="http://maven.apache.org/POM/4.0.0"
xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
<modelVersion>4.0.0</modelVersion>

<parent>
<groupId>org.apache.paimon</groupId>
<artifactId>paimon-service</artifactId>
<version>0.7-SNAPSHOT</version>
</parent>

<artifactId>paimon-service-client</artifactId>
<name>Paimon : Service : Client</name>

<properties>
<flink.shaded.version>17.0</flink.shaded.version>
</properties>

<dependencies>
<dependency>
<groupId>org.apache.paimon</groupId>
<artifactId>paimon-common</artifactId>
<version>${project.version}</version>
<scope>provided</scope>
</dependency>

<dependency>
<groupId>org.apache.paimon</groupId>
<artifactId>paimon-core</artifactId>
<version>${project.version}</version>
<scope>provided</scope>
</dependency>

<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-shaded-netty</artifactId>
<version>4.1.91.Final-${flink.shaded.version}</version>
</dependency>

<dependency>
<groupId>org.apache.paimon</groupId>
<artifactId>paimon-shade-guava-30</artifactId>
<version>${paimon.shade.guava.version}-${paimon.shade.version}</version>
</dependency>
</dependencies>

<build>
<plugins>
<plugin>
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-jar-plugin</artifactId>
<executions>
<execution>
<goals>
<goal>test-jar</goal>
</goals>
</execution>
</executions>
</plugin>
</plugins>
</build>
</project>
Loading

0 comments on commit 9fc5c15

Please sign in to comment.