5757import java .time .Duration ;
5858import java .util .Comparator ;
5959import java .util .List ;
60- import java .util .concurrent .ExecutorService ;
6160import java .util .concurrent .Executors ;
61+ import java .util .concurrent .ScheduledExecutorService ;
62+ import java .util .concurrent .TimeUnit ;
6263
6364@ RunWith (Parameterized .class )
6465public class TestIcebergRESTCatalogGravitinoLlapLocalCliDriver {
6566
67+ private static final CliAdapter CLI_ADAPTER =
68+ new CliConfigs .TestIcebergRESTCatalogGravitinoLlapLocalCliDriver ().getCliAdapter ();
69+
6670 private static final Logger LOG = LoggerFactory .getLogger (TestIcebergRESTCatalogGravitinoLlapLocalCliDriver .class );
6771
6872 private static final String CATALOG_NAME = "ice01" ;
73+ private static final long GRAVITINO_STARTUP_TIMEOUT_MINUTES = 5L ;
6974 private static final String GRAVITINO_CONF_FILE_TEMPLATE = "gravitino-h2-test-template.conf" ;
7075 private static final String GRAVITINO_ROOT_DIR = "/root/gravitino-iceberg-rest-server" ;
71- private static final long GRAVITINO_STARTUP_TIMEOUT_MINUTES = 5L ;
72-
7376 private static final String GRAVITINO_STARTUP_SCRIPT = GRAVITINO_ROOT_DIR + "/bin/start-iceberg-rest-server.sh" ;
7477 private static final String GRAVITINO_H2_LIB = GRAVITINO_ROOT_DIR + "/libs/h2-driver.jar" ;
7578 private static final String GRAVITINO_CONF_FILE = GRAVITINO_ROOT_DIR + "/conf/gravitino-iceberg-rest-server.conf" ;
76-
77- private static final CliAdapter adapter = new CliConfigs .TestIcebergRESTCatalogGravitinoLlapLocalCliDriver ().getCliAdapter ();
7879 private static final DockerImageName GRAVITINO_IMAGE =
79- DockerImageName .parse ("apache/gravitino-iceberg-rest:1.0.0-rc3 " );
80+ DockerImageName .parse ("apache/gravitino-iceberg-rest:1.0.0" );
8081
8182 private final String name ;
8283 private final File qfile ;
8384
8485 private GenericContainer <?> gravitinoContainer ;
8586 private Path warehouseDir ;
86- private final ExecutorService fileSyncExecutor = Executors .newSingleThreadExecutor ();
87+ private final ScheduledExecutorService fileSyncExecutor = Executors .newSingleThreadScheduledExecutor ();
8788
8889 @ Parameters (name = "{0}" )
8990 public static List <Object []> getParameters () throws Exception {
90- return adapter .getParameters ();
91+ return CLI_ADAPTER .getParameters ();
9192 }
9293
9394 @ ClassRule
94- public static final TestRule cliClassRule = adapter .buildClassRule ();
95+ public static final TestRule CLI_CLASS_RULE = CLI_ADAPTER .buildClassRule ();
9596
9697 @ Rule
97- public final TestRule cliTestRule = adapter .buildTestRule ();
98+ public final TestRule cliTestRule = CLI_ADAPTER .buildTestRule ();
9899
99100 public TestIcebergRESTCatalogGravitinoLlapLocalCliDriver (String name , File qfile ) {
100101 this .name = name ;
@@ -111,6 +112,9 @@ public void setup() throws IOException {
111112 String host = gravitinoContainer .getHost ();
112113 Integer port = gravitinoContainer .getMappedPort (9001 );
113114 String restCatalogPrefix = String .format ("%s%s." , CatalogUtils .CATALOG_CONFIG_PREFIX , CATALOG_NAME );
115+
116+ // Suppress IntelliJ warning about using HTTP since this is a local test container connection
117+ @ SuppressWarnings ("HttpUrlsUsage" )
114118 String restCatalogUri = String .format ("http://%s:%d/iceberg" , host , port );
115119
116120 Configuration conf = SessionState .get ().getConf ();
@@ -132,13 +136,40 @@ public void teardown() {
132136 try (var paths = Files .walk (warehouseDir )) {
133137 paths .sorted (Comparator .reverseOrder ())
134138 .forEach (path -> {
135- try { Files .deleteIfExists (path ); }
136- catch (IOException ignored ) {}
139+ try {
140+ Files .deleteIfExists (path );
141+ } catch (IOException e ) {
142+ LOG .debug ("Failed to delete temp file {}" , path , e );
143+ }
137144 });
138- } catch (IOException ignored ) {}
145+ } catch (IOException e ) {
146+ LOG .debug ("Failed to delete temp folder" , e );
147+ }
139148 }
140149 }
141150
151+ /**
152+ * Starts a Gravitino container with the Iceberg REST server configured for testing.
153+ *
154+ * <p>This method configures the container to:
155+ * <ul>
156+ * <li>Expose container REST port 9001 and map it to a host port.</li>
157+ * <li>Modify the container entrypoint to create the warehouse directory before startup.</li>
158+ * <li>Copy a dynamically prepared Gravitino configuration file into the container.</li>
159+ * <li>Copy the H2 driver JAR into the server's lib directory.</li>
160+ * <li>Wait for the Gravitino Iceberg REST server to finish starting (based on logs and port checks).</li>
161+ * <li>Stream container logs into the test logger for easier debugging.</li>
162+ * </ul>
163+ *
164+ * <p>Note: The {@code @SuppressWarnings("resource")} annotation is applied because
165+ * IntelliJ and some compilers flag {@link org.testcontainers.containers.GenericContainer}
166+ * as a resource that should be managed with try-with-resources. In this test setup,
167+ * the container lifecycle is managed explicitly: it is started here and stopped in
168+ * {@code @After} (via {@code gravitinoContainer.stop()}). Using try-with-resources
169+ * would not work in this context, since the container must remain running across
170+ * multiple test methods rather than being confined to a single block scope.</p>
171+ */
172+ @ SuppressWarnings ("resource" )
142173 private void startGravitinoContainer () {
143174 gravitinoContainer = new GenericContainer <>(GRAVITINO_IMAGE )
144175 .withExposedPorts (9001 )
@@ -165,7 +196,8 @@ private void startGravitinoContainer() {
165196 .withStrategy (Wait .forListeningPort ()
166197 .withStartupTimeout (Duration .ofMinutes (GRAVITINO_STARTUP_TIMEOUT_MINUTES )))
167198 )
168- .withLogConsumer (new Slf4jLogConsumer (LoggerFactory .getLogger (TestIcebergRESTCatalogGravitinoLlapLocalCliDriver .class )));
199+ .withLogConsumer (new Slf4jLogConsumer (LoggerFactory
200+ .getLogger (TestIcebergRESTCatalogGravitinoLlapLocalCliDriver .class )));
169201
170202 gravitinoContainer .start ();
171203 }
@@ -190,58 +222,52 @@ private void startGravitinoContainer() {
190222 * to avoid overwriting container data.</p>
191223 */
192224 private void startWarehouseDirSync () {
193- fileSyncExecutor .submit (() -> {
194- try {
195- while (gravitinoContainer .isRunning ()) {
196- try (CopyArchiveFromContainerCmd copyArchiveFromContainerCmd =
197- gravitinoContainer
198- .getDockerClient ()
199- .copyArchiveFromContainerCmd (gravitinoContainer .getContainerId (), warehouseDir .toString ());
200- InputStream tarStream = copyArchiveFromContainerCmd .exec ();
201- TarArchiveInputStream tis = new TarArchiveInputStream (tarStream )) {
202-
203- TarArchiveEntry entry ;
204- while ((entry = tis .getNextTarEntry ()) != null ) {
205- // Skip directories because we only want to copy metadata files from the container.
206- if (entry .isDirectory ()) {
207- continue ;
208- }
209-
210- /*
211- * Tar entry names include a container-specific top-level folder, e.g.:
212- * iceberg-test-1759245909247/iceberg_warehouse/ice_rest/.../metadata.json
213- *
214- * Strip the first part so the relative path inside the warehouse is preserved
215- * when mapping to the host warehouseDir.
216- */
217-
218- String [] parts = entry .getName ().split ("/" , 2 );
219- if (parts .length < 2 ) {
220- continue ; // defensive guard
221- }
225+ fileSyncExecutor .scheduleAtFixedRate (() -> {
226+ if (gravitinoContainer .isRunning ()) {
227+ try (CopyArchiveFromContainerCmd copyArchiveFromContainerCmd =
228+ gravitinoContainer
229+ .getDockerClient ()
230+ .copyArchiveFromContainerCmd (gravitinoContainer .getContainerId (), warehouseDir .toString ());
231+ InputStream tarStream = copyArchiveFromContainerCmd .exec ();
232+ TarArchiveInputStream tis = new TarArchiveInputStream (tarStream )) {
233+
234+ TarArchiveEntry entry ;
235+ while ((entry = tis .getNextEntry ()) != null ) {
236+ // Skip directories because we only want to copy metadata files from the container.
237+ if (entry .isDirectory ()) {
238+ continue ;
239+ }
222240
223- Path relativePath = Paths .get (parts [1 ]);
224- Path outputPath = warehouseDir .resolve (relativePath );
241+ /*
242+ * Tar entry names include a container-specific top-level folder, e.g.:
243+ * iceberg-test-1759245909247/iceberg_warehouse/ice_rest/.../metadata.json
244+ *
245+ * Strip the first part so the relative path inside the warehouse is preserved
246+ * when mapping to the host warehouseDir.
247+ */
248+
249+ String [] parts = entry .getName ().split ("/" , 2 );
250+ if (parts .length < 2 ) {
251+ continue ; // defensive guard
252+ }
225253
226- // Skip if already present on host to avoid overwriting
227- if (Files .exists (outputPath )) {
228- continue ;
229- }
254+ Path relativePath = Paths .get (parts [1 ]);
255+ Path outputPath = warehouseDir .resolve (relativePath );
230256
231- Files .createDirectories (outputPath .getParent ());
232- Files .copy (tis , outputPath );
257+ // Skip if already present on host to avoid overwriting
258+ if (Files .exists (outputPath )) {
259+ continue ;
233260 }
234261
235- } catch ( Exception e ) {
236- LOG . error ( "Warehouse folder sync failed: {}" , e . getMessage () );
262+ Files . createDirectories ( outputPath . getParent ());
263+ Files . copy ( tis , outputPath );
237264 }
238265
239- Thread .sleep (1000 ); // sync every 1 second
266+ } catch (Exception e ) {
267+ LOG .error ("Warehouse folder sync failed: {}" , e .getMessage ());
240268 }
241- } catch (InterruptedException ignored ) {
242- Thread .currentThread ().interrupt ();
243269 }
244- });
270+ }, 0 , 1 , TimeUnit . SECONDS ); // Initial delay: 0, Period: 1 second );
245271 }
246272
247273 private void createWarehouseDir () {
@@ -270,12 +296,6 @@ private void prepareGravitinoConfig() throws IOException {
270296
271297 @ Test
272298 public void testCliDriver () throws Exception {
273- try {
274- adapter .runTest (name , qfile );
275- }
276- catch (Throwable e ) {
277- System .err .println ("Error running " + qfile );
278- throw e ;
279- }
299+ CLI_ADAPTER .runTest (name , qfile );
280300 }
281- }
301+ }
0 commit comments