Skip to content
Merged
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
3 changes: 2 additions & 1 deletion .github/workflows/backend.yml
Original file line number Diff line number Diff line change
Expand Up @@ -1240,7 +1240,8 @@ jobs:
matrix:
java: [ '8', '11' ]
os: [ 'ubuntu-latest' ]
timeout-minutes: 60
# Kudu E2E expands each @TestTemplate case across several PR test containers.
timeout-minutes: 90
steps:
- uses: actions/checkout@v2
- name: Set up JDK ${{ matrix.java }}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -47,11 +47,7 @@
import org.junit.jupiter.api.TestTemplate;
import org.testcontainers.containers.Container;
import org.testcontainers.containers.PulsarContainer;
import org.testcontainers.containers.output.Slf4jLogConsumer;
import org.testcontainers.lifecycle.Startables;
import org.testcontainers.shaded.org.awaitility.Awaitility;
import org.testcontainers.utility.DockerImageName;
import org.testcontainers.utility.DockerLoggerFactory;

import lombok.extern.slf4j.Slf4j;

Expand All @@ -62,7 +58,6 @@
import java.time.Duration;
import java.util.List;
import java.util.concurrent.TimeUnit;
import java.util.stream.Stream;

@Slf4j
public class PulsarBatchIT extends TestSuiteBase implements TestResource {
Expand Down Expand Up @@ -113,15 +108,12 @@ public class PulsarBatchIT extends TestSuiteBase implements TestResource {
@BeforeAll
public void startUp() throws Exception {
pulsarContainer =
new PulsarContainer(DockerImageName.parse(PULSAR_IMAGE_NAME))
.withNetwork(NETWORK)
.withNetworkAliases(PULSAR_HOST)
.withStartupTimeout(Duration.ofMinutes(3))
.withLogConsumer(
new Slf4jLogConsumer(
DockerLoggerFactory.getLogger(PULSAR_IMAGE_NAME)));

Startables.deepStart(Stream.of(pulsarContainer)).join();
PulsarContainerSupport.startPulsarContainer(
dockerClient,
PULSAR_IMAGE_NAME,
NETWORK,
PULSAR_HOST,
Duration.ofMinutes(3));
Awaitility.given()
.ignoreExceptions()
.atLeast(100, TimeUnit.MILLISECONDS)
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,103 @@
/*
* 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.seatunnel.e2e.connector.pulsar;

import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.testcontainers.containers.Network;
import org.testcontainers.containers.PulsarContainer;
import org.testcontainers.containers.output.Slf4jLogConsumer;
import org.testcontainers.lifecycle.Startables;
import org.testcontainers.utility.DockerImageName;
import org.testcontainers.utility.DockerLoggerFactory;

import com.github.dockerjava.api.DockerClient;
import com.github.dockerjava.api.exception.NotFoundException;

import java.time.Duration;
import java.util.concurrent.TimeUnit;
import java.util.stream.Stream;

final class PulsarContainerSupport {

private static final Logger LOG = LoggerFactory.getLogger(PulsarContainerSupport.class);
private static final Duration IMAGE_PULL_TIMEOUT = Duration.ofMinutes(10);

private PulsarContainerSupport() {}

/**
* Pre-pulls the Pulsar image before Testcontainers starts the container. This keeps the first
* CI pull from failing when GitHub runners briefly make no download progress and Testcontainers
* aborts the pull attempt early.
*/
static PulsarContainer startPulsarContainer(
DockerClient dockerClient,
String imageName,
Network network,
String networkAlias,
Duration startupTimeout)
throws InterruptedException {
ensureImageAvailable(dockerClient, imageName);

PulsarContainer pulsarContainer =
new PulsarContainer(DockerImageName.parse(imageName))
.withNetwork(network)
.withNetworkAliases(networkAlias)
.withStartupTimeout(startupTimeout)
.withLogConsumer(
new Slf4jLogConsumer(DockerLoggerFactory.getLogger(imageName)));

Startables.deepStart(Stream.of(pulsarContainer)).join();
return pulsarContainer;
}

private static void ensureImageAvailable(DockerClient dockerClient, String imageName)
throws InterruptedException {
if (isImageAvailable(dockerClient, imageName)) {
LOG.info("Reuse local Pulsar image {}", imageName);
return;
}

DockerImageName dockerImageName = DockerImageName.parse(imageName);
LOG.info(
"Pre-pulling Pulsar image {} with timeout {} to avoid flaky first-pull failures",
imageName,
IMAGE_PULL_TIMEOUT);
boolean completed =
dockerClient
.pullImageCmd(dockerImageName.getUnversionedPart())
.withTag(dockerImageName.getVersionPart())
.start()
.awaitCompletion(IMAGE_PULL_TIMEOUT.getSeconds(), TimeUnit.SECONDS);
if (!completed || !isImageAvailable(dockerClient, imageName)) {
throw new IllegalStateException(
String.format(
"Failed to pre-pull Pulsar image %s within %s",
imageName, IMAGE_PULL_TIMEOUT));
}
}

private static boolean isImageAvailable(DockerClient dockerClient, String imageName) {
try {
dockerClient.inspectImageCmd(imageName).exec();
return true;
} catch (NotFoundException ignored) {
return false;
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -31,19 +31,14 @@
import org.junit.jupiter.api.TestTemplate;
import org.testcontainers.containers.Container;
import org.testcontainers.containers.PulsarContainer;
import org.testcontainers.containers.output.Slf4jLogConsumer;
import org.testcontainers.lifecycle.Startables;
import org.testcontainers.shaded.org.awaitility.Awaitility;
import org.testcontainers.utility.DockerImageName;
import org.testcontainers.utility.DockerLoggerFactory;

import lombok.extern.slf4j.Slf4j;

import java.io.IOException;
import java.nio.charset.StandardCharsets;
import java.time.Duration;
import java.util.concurrent.TimeUnit;
import java.util.stream.Stream;

@Slf4j
public class PulsarMultiTableIT extends TestSuiteBase implements TestResource {
Expand All @@ -60,14 +55,12 @@ public class PulsarMultiTableIT extends TestSuiteBase implements TestResource {
@BeforeAll
public void startUp() throws Exception {
pulsarContainer =
new PulsarContainer(DockerImageName.parse(PULSAR_IMAGE_NAME))
.withNetwork(NETWORK)
.withNetworkAliases(PULSAR_HOST)
.withStartupTimeout(Duration.ofMinutes(3))
.withLogConsumer(
new Slf4jLogConsumer(
DockerLoggerFactory.getLogger(PULSAR_IMAGE_NAME)));
Startables.deepStart(Stream.of(pulsarContainer)).join();
PulsarContainerSupport.startPulsarContainer(
dockerClient,
PULSAR_IMAGE_NAME,
NETWORK,
PULSAR_HOST,
Duration.ofMinutes(3));
Awaitility.given()
.ignoreExceptions()
.atLeast(100, TimeUnit.MILLISECONDS)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -35,11 +35,7 @@
import org.junit.jupiter.api.TestTemplate;
import org.testcontainers.containers.Container;
import org.testcontainers.containers.PulsarContainer;
import org.testcontainers.containers.output.Slf4jLogConsumer;
import org.testcontainers.lifecycle.Startables;
import org.testcontainers.shaded.org.awaitility.Awaitility;
import org.testcontainers.utility.DockerImageName;
import org.testcontainers.utility.DockerLoggerFactory;

import lombok.extern.slf4j.Slf4j;

Expand All @@ -49,7 +45,6 @@
import java.util.List;
import java.util.Random;
import java.util.concurrent.TimeUnit;
import java.util.stream.Stream;

import static java.time.temporal.ChronoUnit.SECONDS;

Expand All @@ -65,15 +60,12 @@ public class PulsarSinkIT extends TestSuiteBase implements TestResource {
@BeforeAll
public void startUp() throws Exception {
pulsarContainer =
new PulsarContainer(DockerImageName.parse(PULSAR_IMAGE_NAME))
.withNetwork(NETWORK)
.withNetworkAliases(PULSAR_HOST)
.withStartupTimeout(Duration.of(400, SECONDS))
.withLogConsumer(
new Slf4jLogConsumer(
DockerLoggerFactory.getLogger(PULSAR_IMAGE_NAME)));

Startables.deepStart(Stream.of(pulsarContainer)).join();
PulsarContainerSupport.startPulsarContainer(
dockerClient,
PULSAR_IMAGE_NAME,
NETWORK,
PULSAR_HOST,
Duration.of(400, SECONDS));
Awaitility.given()
.ignoreExceptions()
.atLeast(100, TimeUnit.MILLISECONDS)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,7 @@

package org.apache.seatunnel.engine.server.trace;

import java.nio.Buffer;
import java.nio.ByteBuffer;
import java.nio.ByteOrder;
import java.util.ArrayList;
Expand Down Expand Up @@ -88,7 +89,11 @@ public static AppendResult append(
}
byte[] newPayload = Arrays.copyOf(payload, payload.length + ENTRY_LENGTH);
ByteBuffer buffer = ByteBuffer.wrap(newPayload).order(ByteOrder.BIG_ENDIAN);
buffer.position(payload.length);
// Cast to Buffer (superclass) so that position(int) resolves to Buffer#position(int)
// returning Buffer, which exists on JDK 8. ByteBuffer overrode this method with a
// covariant ByteBuffer return type only in JDK 9+; calling the ByteBuffer override on
// JDK 8 would throw NoSuchMethodError at runtime.
((Buffer) buffer).position(payload.length);
buffer.put(stage.getCode());
buffer.putLong(taskId);
buffer.putLong(tsMs);
Expand Down
Loading