yashmayya opened a new issue, #14722: URL: https://github.com/apache/pinot/issues/14722
- There are three methods in the `DateTimeUtils` used by the Pinot JDBC client that aren't thread safe - https://github.com/apache/pinot/blob/00f0721662c2e9a3c4a1702bf13dbd937488295a/pinot-clients/pinot-jdbc-client/src/main/java/org/apache/pinot/client/utils/DateTimeUtils.java#L38-L60 - These use the old `SimpleDateFormat` API that is not thread safe and needs external synchronization. The recommended approach is to use the newer (Java 8 and above) `DateTimeFormatter` API that is immutable and thread safe. - These util methods are used in Pinot's JDBC `ResultSet` implementation and causes them to not be thread safe as well. - This issue can easily be reproduced using these simple unit tests: ``` @Test public void testDateFromStringConcurrent() throws Throwable { ExecutorService executorService = Executors.newFixedThreadPool(10); AtomicReference<Throwable> throwable = new AtomicReference<>(); for (int i = 0; i < 10; i++) { executorService.submit(() -> { try { Assert.assertEquals(DateTimeUtils.getDateFromString("2020-01-01", Calendar.getInstance()).toString(), "2020-01-01"); } catch (Throwable t) { throwable.set(t); } }); } executorService.shutdown(); executorService.awaitTermination(1000, TimeUnit.MILLISECONDS); if (throwable.get() != null) { throw throwable.get(); } } @Test public void testTimeFromStringConcurrent() throws Throwable { ExecutorService executorService = Executors.newFixedThreadPool(10); AtomicReference<Throwable> throwable = new AtomicReference<>(); for (int i = 0; i < 10; i++) { executorService.submit(() -> { try { Assert.assertEquals(DateTimeUtils.getTimeFromString("2020-01-01 12:00:00", Calendar.getInstance()).toString(), "12:00:00"); } catch (Throwable t) { throwable.set(t); } }); } executorService.shutdown(); executorService.awaitTermination(1000, TimeUnit.MILLISECONDS); if (throwable.get() != null) { throw throwable.get(); } } @Test public void testTimestampFromStringConcurrent() throws Throwable { ExecutorService executorService = Executors.newFixedThreadPool(10); AtomicReference<Throwable> throwable = new AtomicReference<>(); for (int i = 0; i < 10; i++) { executorService.submit(() -> { try { Assert.assertEquals(DateTimeUtils.getTimestampFromString("2020-01-01 12:00:00", Calendar.getInstance()).toString(), "2020-01-01 12:00:00.0"); } catch (Throwable t) { throwable.set(t); } }); } executorService.shutdown(); executorService.awaitTermination(1000, TimeUnit.MILLISECONDS); if (throwable.get() != null) { throw throwable.get(); } } ``` - We see assertion errors like: ``` java.lang.AssertionError: Expected :2020-01-01 Actual :0202-01-01 ``` - Note that the tests only pass if we make the executor a single threaded one. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: [email protected] For queries about this service, please contact Infrastructure at: [email protected] --------------------------------------------------------------------- To unsubscribe, e-mail: [email protected] For additional commands, e-mail: [email protected]
