From ec5a7123e5ae544248b9dc8ab7b9da818d5ec4af Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ha=CC=8Avard=20Ottestad?= Date: Sun, 19 Jan 2025 14:09:20 +0100 Subject: [PATCH 01/28] ca. 1 800 000 statements per second --- .gitignore | 3 + qendpoint-core/pom.xml | 6 +- .../impl/diskimport/SectionCompressor.java | 7 +- .../iterator/utils/AsyncIteratorFetcher.java | 36 ++- .../iterator/utils/PipedCopyIterator.java | 70 +++++- .../rdf/parsers/ConcurrentInputStream.java | 118 +++++++++ .../core/rdf/parsers/RDFParserRIOT.java | 62 ++++- .../listener/MultiThreadListenerConsole.java | 33 ++- .../core/hdt/impl/HasmacHDTImporterTest.java | 228 ++++++++++++++++++ 9 files changed, 528 insertions(+), 35 deletions(-) create mode 100644 qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/rdf/parsers/ConcurrentInputStream.java create mode 100644 qendpoint-core/src/test/java/com/the_qa_company/qendpoint/core/hdt/impl/HasmacHDTImporterTest.java diff --git a/.gitignore b/.gitignore index deeb8432d..495a5eba9 100755 --- a/.gitignore +++ b/.gitignore @@ -58,3 +58,6 @@ data wikidata qendpoint-store/wdbench-indexes wdbench-results +testing +indexing +wdbench-indexes diff --git a/qendpoint-core/pom.xml b/qendpoint-core/pom.xml index d390abb25..a344da24a 100644 --- a/qendpoint-core/pom.xml +++ b/qendpoint-core/pom.xml @@ -17,8 +17,8 @@ org.apache.maven.plugins maven-compiler-plugin - 17 - 17 + 19 + 19 @@ -75,7 +75,7 @@ org.apache.commons commons-compress - 1.21 + 1.27.1 org.apache.jena diff --git a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/hdt/impl/diskimport/SectionCompressor.java b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/hdt/impl/diskimport/SectionCompressor.java index 05b47197f..09e208473 100644 --- a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/hdt/impl/diskimport/SectionCompressor.java +++ b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/hdt/impl/diskimport/SectionCompressor.java @@ -18,6 +18,7 @@ import com.the_qa_company.qendpoint.core.util.listener.IntermediateListener; import com.the_qa_company.qendpoint.core.util.string.ByteString; import com.the_qa_company.qendpoint.core.util.string.CompactString; +import org.apache.jena.base.Sys; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -51,6 +52,7 @@ public class SectionCompressor implements KWayMerger.KWayMergerImpl source, MultiThreadListener listener, int bufferSize, long chunkSize, int k, boolean debugSleepKwayDict, @@ -250,7 +252,10 @@ public void createChunk(SizeFetcher fetcher, CloseSuppressPath out } if (tripleID % 100_000 == 0) { - listener.notifyProgress(10, "reading triples " + tripleID); + // use start to measure how many triples are read per second + int triplesPerSecond = (int) (tripleID / ((System.currentTimeMillis() - start) / 1000.0)); + + listener.notifyProgress(10, "reading triples " + tripleID + " triples per second: " + triplesPerSecond); } // too much ram allowed? if (subjects.size() == Integer.MAX_VALUE - 6) { diff --git a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/iterator/utils/AsyncIteratorFetcher.java b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/iterator/utils/AsyncIteratorFetcher.java index 788b3ab8a..d0ddca289 100644 --- a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/iterator/utils/AsyncIteratorFetcher.java +++ b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/iterator/utils/AsyncIteratorFetcher.java @@ -1,6 +1,8 @@ package com.the_qa_company.qendpoint.core.iterator.utils; +import java.util.ArrayList; import java.util.Iterator; +import java.util.concurrent.ConcurrentLinkedQueue; import java.util.concurrent.locks.Lock; import java.util.concurrent.locks.ReentrantLock; import java.util.function.Supplier; @@ -15,6 +17,7 @@ public class AsyncIteratorFetcher implements Supplier { private final Iterator iterator; private final Lock lock = new ReentrantLock(); private boolean end; + ConcurrentLinkedQueue queue = new ConcurrentLinkedQueue<>(); public AsyncIteratorFetcher(Iterator iterator) { this.iterator = iterator; @@ -25,16 +28,33 @@ public AsyncIteratorFetcher(Iterator iterator) { */ @Override public E get() { - lock.lock(); - try { - if (iterator.hasNext()) { - return iterator.next(); + E poll = queue.poll(); + + if (poll != null) { + return poll; + } + + synchronized (this) { + poll = queue.poll(); + if (poll == null) { + if (iterator.hasNext()) { + poll = iterator.next(); + } + ArrayList objects = new ArrayList<>(128); + + for (int i = 0; i < 128 && iterator.hasNext(); i++) { + objects.add(iterator.next()); + } + + queue.addAll(objects); + } + + if (poll == null) { + end = true; } - end = true; - return null; - } finally { - lock.unlock(); + return poll; } + } /** diff --git a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/iterator/utils/PipedCopyIterator.java b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/iterator/utils/PipedCopyIterator.java index ab9aa05a0..a6d862ccd 100644 --- a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/iterator/utils/PipedCopyIterator.java +++ b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/iterator/utils/PipedCopyIterator.java @@ -5,6 +5,8 @@ import java.util.Iterator; import java.util.Objects; import java.util.concurrent.ArrayBlockingQueue; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; import java.util.function.Function; /** @@ -110,7 +112,15 @@ public T get() { } } - private final ArrayBlockingQueue> queue = new ArrayBlockingQueue<>(16); + private final ArrayBlockingQueue>[] queue = new ArrayBlockingQueue[] { + new ArrayBlockingQueue<>(16 * 1024), new ArrayBlockingQueue<>(16 * 1024), + new ArrayBlockingQueue<>(16 * 1024), new ArrayBlockingQueue<>(16 * 1024), + new ArrayBlockingQueue<>(16 * 1024), new ArrayBlockingQueue<>(16 * 1024), + new ArrayBlockingQueue<>(16 * 1024), new ArrayBlockingQueue<>(16 * 1024), + new ArrayBlockingQueue<>(16 * 1024), new ArrayBlockingQueue<>(16 * 1024), + new ArrayBlockingQueue<>(16 * 1024), new ArrayBlockingQueue<>(16 * 1024), + new ArrayBlockingQueue<>(16 * 1024), new ArrayBlockingQueue<>(16 * 1024), + new ArrayBlockingQueue<>(16 * 1024), new ArrayBlockingQueue<>(16 * 1024) }; private T next; private boolean end; @@ -118,6 +128,10 @@ public T get() { private Thread thread; + AtomicInteger indexHasNext = new AtomicInteger(0); + + volatile ArrayBlockingQueue> focusQueue; + @Override public boolean hasNext() { if (end) { @@ -129,7 +143,33 @@ public boolean hasNext() { QueueObject obj; try { - obj = queue.take(); + var focusQueue = this.focusQueue; + if (focusQueue != null) { + QueueObject poll = focusQueue.poll(1, TimeUnit.MILLISECONDS); + if (poll != null) { + obj = poll; + } else { + obj = null; + this.focusQueue = null; + } + } else { + obj = null; + } + + if (obj == null) { + + int i = Thread.currentThread().hashCode(); + obj = queue[i % queue.length].poll(10, java.util.concurrent.TimeUnit.MILLISECONDS); + while (obj == null) { + for (ArrayBlockingQueue> queueObjects : queue) { + obj = queueObjects.poll(1, TimeUnit.MILLISECONDS); + if (obj != null) { + break; + } + } + } + } + } catch (InterruptedException e) { throw new PipedIteratorException("Can't read pipe", e); } @@ -162,7 +202,9 @@ public void closePipe() { public void closePipe(Throwable e) { if (e != null) { // clear the queue to force the exception - queue.clear(); + for (ArrayBlockingQueue> queueObjects : queue) { + queueObjects.clear(); + } if (e instanceof PipedIteratorException) { this.exception = (PipedIteratorException) e; } else { @@ -170,7 +212,9 @@ public void closePipe(Throwable e) { } } try { - queue.put(new EndQueueObject()); + for (ArrayBlockingQueue> queueObjects : queue) { + queueObjects.put(new EndQueueObject()); + } } catch (InterruptedException ee) { throw new PipedIteratorException("Can't close pipe", ee); } @@ -198,9 +242,25 @@ public Iterator mapWithId(MapIterator.MapWithIdFunction mappingFunc return new MapIterator<>(this, mappingFunction); } + AtomicInteger index = new AtomicInteger(0); + public void addElement(T node) { + int i = Thread.currentThread().hashCode(); + int l = i % queue.length; try { - queue.put(new ElementQueueObject(node)); + boolean success = queue[l].offer(new ElementQueueObject(node), 10, TimeUnit.MILLISECONDS); + if (!success) { + focusQueue = queue[l]; + while (!success) { + for (ArrayBlockingQueue> queueObjects : queue) { + success = queueObjects.offer(new ElementQueueObject(node), 1, TimeUnit.MILLISECONDS); + if (success) { + break; + } + } + } + } + } catch (InterruptedException ee) { throw new PipedIteratorException("Can't add element to pipe", ee); } diff --git a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/rdf/parsers/ConcurrentInputStream.java b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/rdf/parsers/ConcurrentInputStream.java new file mode 100644 index 000000000..b2574cef4 --- /dev/null +++ b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/rdf/parsers/ConcurrentInputStream.java @@ -0,0 +1,118 @@ +package com.the_qa_company.qendpoint.core.rdf.parsers; + +import java.io.BufferedReader; +import java.io.IOException; +import java.io.InputStream; +import java.io.InputStreamReader; +import java.io.PipedInputStream; +import java.io.PipedOutputStream; +import java.nio.charset.StandardCharsets; + +public class ConcurrentInputStream { + + private final InputStream source; + private final int numberOfStreams; + + private PipedInputStream[] pipedInputStreams; + private PipedOutputStream[] pipedOutputStreams; + + private PipedInputStream bnodeInputStream; + private PipedOutputStream bnodeOutputStream; + + private Thread readerThread; + + public ConcurrentInputStream(InputStream stream, int numberOfStreams) { + this.source = stream; + this.numberOfStreams = numberOfStreams; + setupPipes(); + startReadingThread(); + } + + private void setupPipes() { + pipedInputStreams = new PipedInputStream[numberOfStreams]; + pipedOutputStreams = new PipedOutputStream[numberOfStreams]; + + try { + // Set up main fan-out pipes + for (int i = 0; i < numberOfStreams; i++) { + pipedOutputStreams[i] = new PipedOutputStream(); + pipedInputStreams[i] = new PipedInputStream(pipedOutputStreams[i], 131072 * 1024); + } + + // Set up bnode pipe + bnodeOutputStream = new PipedOutputStream(); + bnodeInputStream = new PipedInputStream(bnodeOutputStream, 131072 * 1024); + + } catch (IOException e) { + throw new RuntimeException("Error creating pipes", e); + } + } + + private void startReadingThread() { + readerThread = new Thread(() -> { + try (BufferedReader reader = new BufferedReader(new InputStreamReader(source, StandardCharsets.UTF_8))) { + String line; + int currentStreamIndex = 0; + long lineCount = 0; + long start = System.currentTimeMillis(); + while ((line = reader.readLine()) != null) { +// lineCount++; +// if (lineCount == 1000000) { +// long end = System.currentTimeMillis(); +// long duration = end - start; +// // print lines per second +// System.out.println(String.format("ConcurrentInputStream lines per second: %,d", +// ((int) Math.floor(lineCount / (duration / 1000.0))))); +// start = end; +// lineCount = 0; +// } + + byte[] data = (line + "\n").getBytes(StandardCharsets.UTF_8); + + if (line.contains("_:")) { + // Write to bnodeOutputStream only + bnodeOutputStream.write(data); + } else { + // Write to a single stream from pipedOutputStreams in a + // round-robin manner + pipedOutputStreams[currentStreamIndex].write(data); + currentStreamIndex = (currentStreamIndex + 1) % pipedOutputStreams.length; + } + } + } catch (IOException e) { + // If there's a read error, close everything. + } finally { + // Close all output streams to signal EOF + for (PipedOutputStream out : pipedOutputStreams) { + try { + out.close(); + } catch (IOException ignored) { + } + } + + try { + bnodeOutputStream.close(); + } catch (IOException ignored) { + } + } + }); + + readerThread.setName("ConcurrentInputStream reader"); + readerThread.setDaemon(true); + readerThread.start(); + } + + /** + * Returns the stream for blank-node lines only. + */ + public InputStream getBnodeStream() { + return bnodeInputStream; + } + + /** + * Returns the array of InputStreams that share all concurrently read data. + */ + public InputStream[] getStreams() { + return pipedInputStreams; + } +} diff --git a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/rdf/parsers/RDFParserRIOT.java b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/rdf/parsers/RDFParserRIOT.java index a89ec1e12..450c8ad39 100644 --- a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/rdf/parsers/RDFParserRIOT.java +++ b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/rdf/parsers/RDFParserRIOT.java @@ -20,6 +20,8 @@ import java.io.FileNotFoundException; import java.io.InputStream; +import java.util.ArrayList; +import java.util.concurrent.atomic.AtomicInteger; import com.the_qa_company.qendpoint.core.quad.QuadString; import org.apache.jena.graph.Triple; @@ -44,9 +46,47 @@ public class RDFParserRIOT implements RDFParserCallback { private static final Logger log = LoggerFactory.getLogger(RDFParserRIOT.class); private void parse(InputStream stream, String baseUri, Lang lang, boolean keepBNode, ElemStringBuffer buffer) { + Thread.dumpStack(); + if (keepBNode) { - RDFParser.source(stream).base(baseUri).lang(lang).labelToNode(LabelToNode.createUseLabelAsGiven()) - .parse(buffer); + ConcurrentInputStream cs = new ConcurrentInputStream(stream, 11); + + InputStream bnodes = cs.getBnodeStream(); + + var threads = new ArrayList(); + + Thread e1 = new Thread(() -> { + RDFParser.source(bnodes).base(baseUri).lang(lang).labelToNode(LabelToNode.createUseLabelAsGiven()) + .parse(buffer); + }); + e1.setName("BNode parser"); + threads.add(e1); + + InputStream[] streams = cs.getStreams(); + int i = 0; + for (InputStream s : streams) { + int temp = i + 1; + Thread e = new Thread(() -> { + RDFParser.source(s).base(baseUri).lang(lang).labelToNode(LabelToNode.createUseLabelAsGiven()) + .parse(buffer); + }); + i++; + e.setName("Stream parser " + i); + threads.add(e); + + } + + threads.forEach(Thread::start); + for (Thread thread : threads) { + try { + thread.join(); + } catch (InterruptedException e) { + throw new RuntimeException(e); + } + } + +// RDFParser.source(stream).base(baseUri).lang(lang).labelToNode(LabelToNode.createUseLabelAsGiven()) +// .parse(buffer); } else { RDFParser.source(stream).base(baseUri).lang(lang).parse(buffer); } @@ -75,14 +115,13 @@ public void doParse(String fileName, String baseUri, RDFNotation notation, boole public void doParse(InputStream input, String baseUri, RDFNotation notation, boolean keepBNode, RDFCallback callback) throws ParserException { try { - ElemStringBuffer buffer = new ElemStringBuffer(callback); switch (notation) { - case NTRIPLES -> parse(input, baseUri, Lang.NTRIPLES, keepBNode, buffer); - case NQUAD -> parse(input, baseUri, Lang.NQUADS, keepBNode, buffer); - case RDFXML -> parse(input, baseUri, Lang.RDFXML, keepBNode, buffer); - case N3, TURTLE -> parse(input, baseUri, Lang.TURTLE, keepBNode, buffer); - case TRIG -> parse(input, baseUri, Lang.TRIG, keepBNode, buffer); - case TRIX -> parse(input, baseUri, Lang.TRIX, keepBNode, buffer); + case NTRIPLES -> parse(input, baseUri, Lang.NTRIPLES, keepBNode, new ElemStringBuffer(callback)); + case NQUAD -> parse(input, baseUri, Lang.NQUADS, keepBNode, new ElemStringBuffer(callback)); + case RDFXML -> parse(input, baseUri, Lang.RDFXML, keepBNode, new ElemStringBuffer(callback)); + case N3, TURTLE -> parse(input, baseUri, Lang.TURTLE, keepBNode, new ElemStringBuffer(callback)); + case TRIG -> parse(input, baseUri, Lang.TRIG, keepBNode, new ElemStringBuffer(callback)); + case TRIX -> parse(input, baseUri, Lang.TRIX, keepBNode, new ElemStringBuffer(callback)); default -> throw new NotImplementedException("Parser not found for format " + notation); } } catch (Exception e) { @@ -91,12 +130,13 @@ public void doParse(InputStream input, String baseUri, RDFNotation notation, boo } } - private static class ElemStringBuffer implements StreamRDF { + public static class ElemStringBuffer implements StreamRDF { private final TripleString triple = new TripleString(); private final QuadString quad = new QuadString(); private final RDFCallback callback; + private final static AtomicInteger counter = new AtomicInteger(0); - private ElemStringBuffer(RDFCallback callback) { + public ElemStringBuffer(RDFCallback callback) { this.callback = callback; } diff --git a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/util/listener/MultiThreadListenerConsole.java b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/util/listener/MultiThreadListenerConsole.java index a6e78f838..1b28f415f 100644 --- a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/util/listener/MultiThreadListenerConsole.java +++ b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/util/listener/MultiThreadListenerConsole.java @@ -2,6 +2,8 @@ import java.util.Map; import java.util.TreeMap; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; import com.the_qa_company.qendpoint.core.listener.MultiThreadListener; @@ -53,11 +55,11 @@ public MultiThreadListenerConsole(boolean color) { public MultiThreadListenerConsole(boolean color, boolean asciiListener) { this.color = color || ALLOW_COLOR_SEQUENCE; - if (asciiListener) { - threadMessages = new TreeMap<>(); - } else { - threadMessages = null; - } +// if (asciiListener) { + threadMessages = new TreeMap<>(); +// } else { +// threadMessages = null; +// } } public String color(int r, int g, int b) { @@ -140,7 +142,7 @@ public synchronized void notifyProgress(String thread, float level, String messa String msg = colorReset() + progressBar(level) + colorReset() + " " + message; if (threadMessages != null) { threadMessages.put(thread, msg); - render(); +// render(); } else { System.out.println(colorReset() + "[" + colorThread() + thread + colorReset() + "]" + msg); } @@ -160,11 +162,27 @@ public void removeLast() { System.out.print(message); } + { + ExecutorService executorService = Executors.newSingleThreadExecutor(); + Executors.newSingleThreadExecutor().submit(() -> { + while (true) { + try { + Thread.sleep(500); + render(); + } catch (InterruptedException e) { + break; + } + executorService.shutdown(); + } + }); + + } + private void render() { render(null); } - private void render(String ln) { + synchronized private void render(String ln) { if (threadMessages == null) { return; } @@ -197,5 +215,6 @@ private void render(String ln) { previous = lines; System.out.print(message); + System.out.flush(); } } diff --git a/qendpoint-core/src/test/java/com/the_qa_company/qendpoint/core/hdt/impl/HasmacHDTImporterTest.java b/qendpoint-core/src/test/java/com/the_qa_company/qendpoint/core/hdt/impl/HasmacHDTImporterTest.java new file mode 100644 index 000000000..f3262fa3f --- /dev/null +++ b/qendpoint-core/src/test/java/com/the_qa_company/qendpoint/core/hdt/impl/HasmacHDTImporterTest.java @@ -0,0 +1,228 @@ +package com.the_qa_company.qendpoint.core.hdt.impl; + +import com.the_qa_company.qendpoint.core.enums.CompressionType; +import com.the_qa_company.qendpoint.core.enums.RDFNotation; +import com.the_qa_company.qendpoint.core.exceptions.ParserException; +import com.the_qa_company.qendpoint.core.iterator.utils.PipedCopyIterator; +import com.the_qa_company.qendpoint.core.options.HDTSpecification; +import com.the_qa_company.qendpoint.core.rdf.RDFParserCallback; +import com.the_qa_company.qendpoint.core.rdf.RDFParserFactory; +import com.the_qa_company.qendpoint.core.rdf.parsers.ConcurrentInputStream; +import com.the_qa_company.qendpoint.core.rdf.parsers.RDFParserRIOT; +import com.the_qa_company.qendpoint.core.triples.TripleString; +import com.the_qa_company.qendpoint.core.triples.impl.utils.HDTTestUtils; +import com.the_qa_company.qendpoint.core.util.io.IOUtil; +import org.apache.jena.riot.Lang; +import org.apache.jena.riot.RDFParser; +import org.apache.jena.riot.lang.LabelToNode; +import org.junit.Test; + +import java.io.BufferedReader; +import java.io.FileInputStream; +import java.io.IOException; +import java.io.InputStream; +import java.io.InputStreamReader; +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; +import java.util.concurrent.atomic.LongAdder; + +import static org.apache.jena.riot.lang.extra.TurtleJCC.lang; + +public class HasmacHDTImporterTest { + + private final HDTSpecification spec; + + public HasmacHDTImporterTest() { + spec = new HDTSpecification(); + spec.set("loader.type", "one-pass"); + spec.set("loader.bnode.seed", "1234567"); + } + + private Iterator asIt(String file) throws ParserException { + List triples = new ArrayList<>(); + RDFNotation notation = RDFNotation.guess(file); + RDFParserCallback parser = RDFParserFactory.getParserCallback(notation); + parser.doParse(file, HDTTestUtils.BASE_URI, notation, true, (triple, pos) -> { + // force duplication of the triple string data + triples.add(new TripleString(triple.getSubject().toString(), triple.getPredicate().toString(), + triple.getObject().toString())); + }); + return triples.iterator(); + } + + @Test + public void testGz() throws ParserException, IOException { + FileInputStream fileStream = new FileInputStream( + "/Users/havardottestad/Documents/Programming/qEndpoint2/indexing/latest-truthy.nt.gz"); + + try (InputStream uncompressed = IOUtil.asUncompressed(fileStream, CompressionType.GZIP); + BufferedReader reader = new BufferedReader(new InputStreamReader(uncompressed))) { + long sum = 0; + String line; + long startTime = System.currentTimeMillis(); + int lineCount = 0; + int checkpoint = 1000000; + + while ((line = reader.readLine()) != null) { + sum += line.length(); + lineCount++; + if (lineCount == checkpoint) { + long currentTime = System.currentTimeMillis(); + long elapsedTime = currentTime - startTime; // in + // milliseconds + int linesPerSecond = ((int) Math.floor(checkpoint / (elapsedTime / 1000.0))); + + // TODO: print linesPerSecond with thousands separator + System.out.println(String.format("Lines per second: %,d", linesPerSecond)); + + startTime = currentTime; // reset start time for the next + // checkpoint + lineCount = 0; // reset line count for the next checkpoint + } + } + System.out.println(sum); + } + } + + @Test + public void concurentInputStreamTest() throws ParserException, IOException { + try (InputStream fileStream = new FileInputStream( + "/Users/havardottestad/Documents/Programming/qEndpoint2/indexing/latest-truthy.nt.gz")) { + + InputStream uncompressed = IOUtil.asUncompressed(fileStream, CompressionType.GZIP); + + LongAdder longAdder = new LongAdder(); + + ConcurrentInputStream cs = new ConcurrentInputStream(uncompressed, 10); + + InputStream bnodes = cs.getBnodeStream(); + + var threads = new ArrayList(); + + Thread e1 = new Thread(() -> { + BufferedReader reader = new BufferedReader(new InputStreamReader(bnodes)); + while (true) { + try { + if (!(reader.readLine() != null)) + break; + } catch (IOException e) { + throw new RuntimeException(e); + } + longAdder.increment(); + } + + }); + e1.setName("BNode parser"); + threads.add(e1); + + InputStream[] streams = cs.getStreams(); + int i = 0; + for (InputStream s : streams) { + int temp = i + 1; + Thread e = new Thread(() -> { + BufferedReader reader = new BufferedReader(new InputStreamReader(s)); + while (true) { + try { + if (!(reader.readLine() != null)) + break; + } catch (IOException e2) { + throw new RuntimeException(e2); + } + longAdder.increment(); + } + }); + i++; + e.setName("Stream parser " + i); + threads.add(e); + + } + + threads.forEach(Thread::start); + + new Thread(() -> { + while (true) { + try { + Thread.sleep(10 * 1000); + } catch (InterruptedException e) { + e.printStackTrace(); + } + System.out.println(String.format("Lines per second: %,d", longAdder.sumThenReset() / 10)); + + } + }).start(); + + for (Thread thread : threads) { + try { + thread.join(); + } catch (InterruptedException e) { + throw new RuntimeException(e); + } + } + } + + } + + @Test + public void concurrentParsing() throws ParserException, IOException { + try (InputStream fileStream = new FileInputStream( + "/Users/havardottestad/Documents/Programming/qEndpoint2/indexing/latest-truthy.nt.gz")) { + + InputStream uncompressed = IOUtil.asUncompressed(fileStream, CompressionType.GZIP); + + LongAdder longAdder = new LongAdder(); + + ConcurrentInputStream cs = new ConcurrentInputStream(uncompressed, 11); + + InputStream bnodes = cs.getBnodeStream(); + + var threads = new ArrayList(); + + RDFParser parser1 = RDFParser.source(bnodes).base("").lang(Lang.NTRIPLES) + .labelToNode(LabelToNode.createUseLabelAsGiven()).build(); + Thread e1 = new Thread(() -> { + parser1.parse(new RDFParserRIOT.ElemStringBuffer((triple, pos) -> longAdder.increment())); + }); + e1.setName("BNode parser"); + threads.add(e1); + + InputStream[] streams = cs.getStreams(); + int i = 0; + for (InputStream s : streams) { + int temp = i + 1; + RDFParser parser = RDFParser.source(s).base("").lang(Lang.NTRIPLES) + .labelToNode(LabelToNode.createUseLabelAsGiven()).build(); + Thread e = new Thread(() -> { + parser.parse(new RDFParserRIOT.ElemStringBuffer((triple, pos) -> longAdder.increment())); + }); + i++; + e.setName("Stream parser " + i); + threads.add(e); + + } + + threads.forEach(Thread::start); + + new Thread(() -> { + while (true) { + try { + Thread.sleep(10 * 1000); + } catch (InterruptedException e) { + e.printStackTrace(); + } + System.out.println(String.format("Lines per second: %,d", longAdder.sumThenReset() / 10)); + } + }).start(); + + for (Thread thread : threads) { + try { + thread.join(); + } catch (InterruptedException e) { + throw new RuntimeException(e); + } + } + } + + } + +} From bd52df3813974033fa7dfda004ca00abf31af7e3 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ha=CC=8Avard=20Ottestad?= Date: Sun, 19 Jan 2025 16:00:05 +0100 Subject: [PATCH 02/28] ca. 1 800 000 statements per second --- results.md | 4 ++++ 1 file changed, 4 insertions(+) create mode 100644 results.md diff --git a/results.md b/results.md new file mode 100644 index 000000000..19c6d11b4 --- /dev/null +++ b/results.md @@ -0,0 +1,4 @@ +testGz: 6,134,969 lines per second +concurentInputStreamTest: 3,832,601 lines per second +concurrentParsing: 3,649,347 lines per second +rdf2hdt.sh: 1,839,333 lines per second From 210e1d2bd8bba3edccb712b38d18dfbf5401e4a9 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ha=CC=8Avard=20Ottestad?= Date: Sun, 19 Jan 2025 16:19:50 +0100 Subject: [PATCH 03/28] ca. 1 950 000 triples per second --- .../qendpoint/core/iterator/utils/AsyncIteratorFetcher.java | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/iterator/utils/AsyncIteratorFetcher.java b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/iterator/utils/AsyncIteratorFetcher.java index d0ddca289..4696c3cb5 100644 --- a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/iterator/utils/AsyncIteratorFetcher.java +++ b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/iterator/utils/AsyncIteratorFetcher.java @@ -14,6 +14,7 @@ * @author Antoine Willerval */ public class AsyncIteratorFetcher implements Supplier { + public static final int BUFFER = 1024; private final Iterator iterator; private final Lock lock = new ReentrantLock(); private boolean end; @@ -28,6 +29,7 @@ public AsyncIteratorFetcher(Iterator iterator) { */ @Override public E get() { + E poll = queue.poll(); if (poll != null) { @@ -40,9 +42,9 @@ public E get() { if (iterator.hasNext()) { poll = iterator.next(); } - ArrayList objects = new ArrayList<>(128); + ArrayList objects = new ArrayList<>(BUFFER); - for (int i = 0; i < 128 && iterator.hasNext(); i++) { + for (int i = 0; i < BUFFER && iterator.hasNext(); i++) { objects.add(iterator.next()); } From b3a2190b234661719e6ee0de49fc7352e13c06f0 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ha=CC=8Avard=20Ottestad?= Date: Sun, 19 Jan 2025 16:20:22 +0100 Subject: [PATCH 04/28] ca. 1 950 000 triples per second --- results.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/results.md b/results.md index 19c6d11b4..93fd05c85 100644 --- a/results.md +++ b/results.md @@ -1,4 +1,4 @@ testGz: 6,134,969 lines per second concurentInputStreamTest: 3,832,601 lines per second concurrentParsing: 3,649,347 lines per second -rdf2hdt.sh: 1,839,333 lines per second +rdf2hdt.sh: 1,974,133 lines per second From b6330368ae4cd67e9ae1f76a38674d9859b34509 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ha=CC=8Avard=20Ottestad?= Date: Sun, 19 Jan 2025 16:27:02 +0100 Subject: [PATCH 05/28] ca. 1 950 000 triples per second --- .../core/rdf/parsers/ConcurrentInputStream.java | 10 ++++++++-- 1 file changed, 8 insertions(+), 2 deletions(-) diff --git a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/rdf/parsers/ConcurrentInputStream.java b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/rdf/parsers/ConcurrentInputStream.java index b2574cef4..03674807c 100644 --- a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/rdf/parsers/ConcurrentInputStream.java +++ b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/rdf/parsers/ConcurrentInputStream.java @@ -32,16 +32,22 @@ private void setupPipes() { pipedInputStreams = new PipedInputStream[numberOfStreams]; pipedOutputStreams = new PipedOutputStream[numberOfStreams]; + // The size of the pipes needs to be larger than the buffer of the + // buffered reader that Jena uses inside the parser, which is 131072 + // bytes. If our pipeSize is too small it limits the ability for the + // parsers to work concurrently. + int pipeSize = 131072 * 1024; + try { // Set up main fan-out pipes for (int i = 0; i < numberOfStreams; i++) { pipedOutputStreams[i] = new PipedOutputStream(); - pipedInputStreams[i] = new PipedInputStream(pipedOutputStreams[i], 131072 * 1024); + pipedInputStreams[i] = new PipedInputStream(pipedOutputStreams[i], pipeSize); } // Set up bnode pipe bnodeOutputStream = new PipedOutputStream(); - bnodeInputStream = new PipedInputStream(bnodeOutputStream, 131072 * 1024); + bnodeInputStream = new PipedInputStream(bnodeOutputStream, pipeSize); } catch (IOException e) { throw new RuntimeException("Error creating pipes", e); From f57af209a58a716df41dc3fb4693b7c6f0e70438 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ha=CC=8Avard=20Ottestad?= Date: Mon, 20 Jan 2025 11:03:15 +0100 Subject: [PATCH 06/28] ca. 2 050 000 after stabilisation --- qendpoint-core/pom.xml | 4 ++-- .../core/iterator/utils/AsyncIteratorFetcher.java | 13 +++++++++---- .../core/iterator/utils/PipedCopyIterator.java | 6 +++++- 3 files changed, 16 insertions(+), 7 deletions(-) diff --git a/qendpoint-core/pom.xml b/qendpoint-core/pom.xml index a344da24a..f0df8216e 100644 --- a/qendpoint-core/pom.xml +++ b/qendpoint-core/pom.xml @@ -17,8 +17,8 @@ org.apache.maven.plugins maven-compiler-plugin - 19 - 19 + 17 + 17 diff --git a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/iterator/utils/AsyncIteratorFetcher.java b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/iterator/utils/AsyncIteratorFetcher.java index 4696c3cb5..572a719b0 100644 --- a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/iterator/utils/AsyncIteratorFetcher.java +++ b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/iterator/utils/AsyncIteratorFetcher.java @@ -1,7 +1,9 @@ package com.the_qa_company.qendpoint.core.iterator.utils; +import java.util.ArrayDeque; import java.util.ArrayList; import java.util.Iterator; +import java.util.Queue; import java.util.concurrent.ConcurrentLinkedQueue; import java.util.concurrent.locks.Lock; import java.util.concurrent.locks.ReentrantLock; @@ -18,7 +20,8 @@ public class AsyncIteratorFetcher implements Supplier { private final Iterator iterator; private final Lock lock = new ReentrantLock(); private boolean end; - ConcurrentLinkedQueue queue = new ConcurrentLinkedQueue<>(); + Queue[] queue = new Queue[] { new ConcurrentLinkedQueue<>(), new ConcurrentLinkedQueue<>(), + new ConcurrentLinkedQueue<>(), new ConcurrentLinkedQueue<>() }; public AsyncIteratorFetcher(Iterator iterator) { this.iterator = iterator; @@ -30,14 +33,16 @@ public AsyncIteratorFetcher(Iterator iterator) { @Override public E get() { - E poll = queue.poll(); + int index = (int) (Thread.currentThread().getId() % queue.length); + + E poll = queue[index].poll(); if (poll != null) { return poll; } synchronized (this) { - poll = queue.poll(); + poll = queue[index].poll(); if (poll == null) { if (iterator.hasNext()) { poll = iterator.next(); @@ -48,7 +53,7 @@ public E get() { objects.add(iterator.next()); } - queue.addAll(objects); + queue[index].addAll(objects); } if (poll == null) { diff --git a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/iterator/utils/PipedCopyIterator.java b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/iterator/utils/PipedCopyIterator.java index a6d862ccd..560b33f07 100644 --- a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/iterator/utils/PipedCopyIterator.java +++ b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/iterator/utils/PipedCopyIterator.java @@ -120,7 +120,11 @@ public T get() { new ArrayBlockingQueue<>(16 * 1024), new ArrayBlockingQueue<>(16 * 1024), new ArrayBlockingQueue<>(16 * 1024), new ArrayBlockingQueue<>(16 * 1024), new ArrayBlockingQueue<>(16 * 1024), new ArrayBlockingQueue<>(16 * 1024), - new ArrayBlockingQueue<>(16 * 1024), new ArrayBlockingQueue<>(16 * 1024) }; + new ArrayBlockingQueue<>(16 * 1024), new ArrayBlockingQueue<>(16 * 1024), + new ArrayBlockingQueue<>(16 * 1024), new ArrayBlockingQueue<>(16 * 1024), + new ArrayBlockingQueue<>(16 * 1024), new ArrayBlockingQueue<>(16 * 1024), + new ArrayBlockingQueue<>(16 * 1024), new ArrayBlockingQueue<>(16 * 1024), + new ArrayBlockingQueue<>(16 * 1024), new ArrayBlockingQueue<>(16 * 1024), }; private T next; private boolean end; From 76972255767fb07226332e05bf75927c190116f6 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ha=CC=8Avard=20Ottestad?= Date: Mon, 20 Jan 2025 12:30:42 +0100 Subject: [PATCH 07/28] ca. 2 300 000 after stabilisation --- .../iterator/utils/AsyncIteratorFetcher.java | 53 ++++++++------ .../iterator/utils/PipedCopyIterator.java | 71 +++++++++++++------ 2 files changed, 78 insertions(+), 46 deletions(-) diff --git a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/iterator/utils/AsyncIteratorFetcher.java b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/iterator/utils/AsyncIteratorFetcher.java index 572a719b0..36879b90f 100644 --- a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/iterator/utils/AsyncIteratorFetcher.java +++ b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/iterator/utils/AsyncIteratorFetcher.java @@ -4,7 +4,6 @@ import java.util.ArrayList; import java.util.Iterator; import java.util.Queue; -import java.util.concurrent.ConcurrentLinkedQueue; import java.util.concurrent.locks.Lock; import java.util.concurrent.locks.ReentrantLock; import java.util.function.Supplier; @@ -16,12 +15,15 @@ * @author Antoine Willerval */ public class AsyncIteratorFetcher implements Supplier { - public static final int BUFFER = 1024; + public static final int BUFFER = 1024 * 4; private final Iterator iterator; private final Lock lock = new ReentrantLock(); private boolean end; - Queue[] queue = new Queue[] { new ConcurrentLinkedQueue<>(), new ConcurrentLinkedQueue<>(), - new ConcurrentLinkedQueue<>(), new ConcurrentLinkedQueue<>() }; + Queue[] queue = new Queue[] { new ArrayDeque(BUFFER), new ArrayDeque(BUFFER), new ArrayDeque(BUFFER), + new ArrayDeque(BUFFER), new ArrayDeque(BUFFER), new ArrayDeque(BUFFER), new ArrayDeque(BUFFER), + new ArrayDeque(BUFFER), new ArrayDeque(BUFFER), new ArrayDeque(BUFFER), new ArrayDeque(BUFFER), + new ArrayDeque(BUFFER), new ArrayDeque(BUFFER), new ArrayDeque(BUFFER), new ArrayDeque(BUFFER), + new ArrayDeque(BUFFER), }; public AsyncIteratorFetcher(Iterator iterator) { this.iterator = iterator; @@ -35,31 +37,36 @@ public E get() { int index = (int) (Thread.currentThread().getId() % queue.length); - E poll = queue[index].poll(); + // With this approach there is some risk that a queue is filled but + // never emptied. Maybe we should look for another queue to read from + // before filling our own queue? + synchronized (queue[index]) { + E poll = queue[index].poll(); - if (poll != null) { - return poll; - } + if (poll != null) { + return poll; + } - synchronized (this) { - poll = queue[index].poll(); - if (poll == null) { - if (iterator.hasNext()) { - poll = iterator.next(); - } - ArrayList objects = new ArrayList<>(BUFFER); + synchronized (this) { + poll = queue[index].poll(); + if (poll == null) { + if (iterator.hasNext()) { + poll = iterator.next(); + } + ArrayList objects = new ArrayList<>(BUFFER); - for (int i = 0; i < BUFFER && iterator.hasNext(); i++) { - objects.add(iterator.next()); - } + for (int i = 0; i < BUFFER && iterator.hasNext(); i++) { + objects.add(iterator.next()); + } - queue[index].addAll(objects); - } + queue[index].addAll(objects); + } - if (poll == null) { - end = true; + if (poll == null) { + end = true; + } + return poll; } - return poll; } } diff --git a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/iterator/utils/PipedCopyIterator.java b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/iterator/utils/PipedCopyIterator.java index 560b33f07..1343e0536 100644 --- a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/iterator/utils/PipedCopyIterator.java +++ b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/iterator/utils/PipedCopyIterator.java @@ -147,31 +147,11 @@ public boolean hasNext() { QueueObject obj; try { - var focusQueue = this.focusQueue; - if (focusQueue != null) { - QueueObject poll = focusQueue.poll(1, TimeUnit.MILLISECONDS); - if (poll != null) { - obj = poll; - } else { - obj = null; - this.focusQueue = null; - } - } else { - obj = null; - } + obj = useFocusQueue(); if (obj == null) { - int i = Thread.currentThread().hashCode(); - obj = queue[i % queue.length].poll(10, java.util.concurrent.TimeUnit.MILLISECONDS); - while (obj == null) { - for (ArrayBlockingQueue> queueObjects : queue) { - obj = queueObjects.poll(1, TimeUnit.MILLISECONDS); - if (obj != null) { - break; - } - } - } + obj = useThreadBasedQueue(); } } catch (InterruptedException e) { @@ -189,6 +169,51 @@ public boolean hasNext() { return true; } + private QueueObject useThreadBasedQueue() throws InterruptedException { + QueueObject obj; + int i = Thread.currentThread().hashCode(); + obj = queue[i % queue.length].poll(); + if (obj == null) { + obj = iterateThroughAllQueues(obj); + } else if (focusQueue == null) { + focusQueue = queue[i % queue.length]; + } + return obj; + } + + private QueueObject iterateThroughAllQueues(QueueObject obj) throws InterruptedException { + while (obj == null) { + for (ArrayBlockingQueue> queueObjects : queue) { + obj = queueObjects.poll(); + if (obj != null) { + if (focusQueue == null) { + focusQueue = queueObjects; + } + return obj; + } + } + Thread.sleep(10); + } + return obj; + } + + private QueueObject useFocusQueue() throws InterruptedException { + QueueObject obj; + var focusQueue = this.focusQueue; + if (focusQueue != null) { + QueueObject poll = focusQueue.poll(); + if (poll != null) { + obj = poll; + } else { + obj = null; + this.focusQueue = null; + } + } else { + obj = null; + } + return obj; + } + @Override public T next() { if (!hasNext()) { @@ -252,7 +277,7 @@ public void addElement(T node) { int i = Thread.currentThread().hashCode(); int l = i % queue.length; try { - boolean success = queue[l].offer(new ElementQueueObject(node), 10, TimeUnit.MILLISECONDS); + boolean success = queue[l].offer(new ElementQueueObject(node)); if (!success) { focusQueue = queue[l]; while (!success) { From b9b4d799b34e4af5998a98e93a6f1fb9438b4d71 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ha=CC=8Avard=20Ottestad?= Date: Mon, 20 Jan 2025 18:32:47 +0100 Subject: [PATCH 08/28] faster sorting and compression --- .../core/hdt/impl/HDTDiskImporter.java | 13 ++- .../iterator/utils/AsyncIteratorFetcher.java | 50 ++---------- .../utils/AsyncIteratorFetcherUnordered.java | 81 +++++++++++++++++++ .../core/rdf/parsers/RDFParserRIOT.java | 5 +- .../listener/MultiThreadListenerConsole.java | 9 ++- .../core/util/string/ByteString.java | 26 +++--- .../core/util/string/ByteStringUtil.java | 15 ++++ 7 files changed, 140 insertions(+), 59 deletions(-) create mode 100644 qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/iterator/utils/AsyncIteratorFetcherUnordered.java diff --git a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/hdt/impl/HDTDiskImporter.java b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/hdt/impl/HDTDiskImporter.java index 5b20e8378..fe03550bd 100644 --- a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/hdt/impl/HDTDiskImporter.java +++ b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/hdt/impl/HDTDiskImporter.java @@ -13,6 +13,7 @@ import com.the_qa_company.qendpoint.core.hdt.impl.diskimport.TripleCompressionResult; import com.the_qa_company.qendpoint.core.header.HeaderPrivate; import com.the_qa_company.qendpoint.core.iterator.utils.AsyncIteratorFetcher; +import com.the_qa_company.qendpoint.core.iterator.utils.AsyncIteratorFetcherUnordered; import com.the_qa_company.qendpoint.core.listener.MultiThreadListener; import com.the_qa_company.qendpoint.core.listener.ProgressListener; import com.the_qa_company.qendpoint.core.options.HDTOptions; @@ -100,10 +101,14 @@ public HDTDiskImporter(HDTOptions hdtFormat, ProgressListener progressListener, throw new IllegalArgumentException("Number of workers should be positive!"); } // maximum size of a chunk - chunkSize = hdtFormat.getInt(HDTOptionsKeys.LOADER_DISK_CHUNK_SIZE_KEY, () -> getMaxChunkSize(this.workers)); + long chunkSize = hdtFormat.getInt(HDTOptionsKeys.LOADER_DISK_CHUNK_SIZE_KEY, + () -> getMaxChunkSize(this.workers)); if (chunkSize < 0) { throw new IllegalArgumentException("Negative chunk size!"); } + System.err.println("chunkSize: " + chunkSize); + this.chunkSize = ((((chunkSize / 1024 / 1024) / 32) * 32) * 1024 * 1024); + System.err.println("this.chunkSize: " + this.chunkSize); long maxFileOpenedLong = hdtFormat.getInt(HDTOptionsKeys.LOADER_DISK_MAX_FILE_OPEN_KEY, 1024); int maxFileOpened; if (maxFileOpenedLong < 0 || maxFileOpenedLong > Integer.MAX_VALUE) { @@ -178,10 +183,10 @@ public CompressTripleMapper compressDictionary(Iterator iterator) throw new IllegalArgumentException("Dictionary already built! Use another importer instance!"); } listener.notifyProgress(0, - "Sorting sections with chunk of size: " + StringUtil.humanReadableByteCount(chunkSize, true) + "B with " - + ways + "ways and " + workers + " worker(s)"); + "Sorting sections with chunk of size: " + StringUtil.humanReadableByteCount(chunkSize, false) + + "iB with " + ways + "ways and " + workers + " worker(s)"); - AsyncIteratorFetcher source = new AsyncIteratorFetcher<>(iterator); + AsyncIteratorFetcherUnordered source = new AsyncIteratorFetcherUnordered<>(iterator); profiler.pushSection("section compression"); CompressionResult compressionResult; diff --git a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/iterator/utils/AsyncIteratorFetcher.java b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/iterator/utils/AsyncIteratorFetcher.java index 36879b90f..788b3ab8a 100644 --- a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/iterator/utils/AsyncIteratorFetcher.java +++ b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/iterator/utils/AsyncIteratorFetcher.java @@ -1,9 +1,6 @@ package com.the_qa_company.qendpoint.core.iterator.utils; -import java.util.ArrayDeque; -import java.util.ArrayList; import java.util.Iterator; -import java.util.Queue; import java.util.concurrent.locks.Lock; import java.util.concurrent.locks.ReentrantLock; import java.util.function.Supplier; @@ -15,15 +12,9 @@ * @author Antoine Willerval */ public class AsyncIteratorFetcher implements Supplier { - public static final int BUFFER = 1024 * 4; private final Iterator iterator; private final Lock lock = new ReentrantLock(); private boolean end; - Queue[] queue = new Queue[] { new ArrayDeque(BUFFER), new ArrayDeque(BUFFER), new ArrayDeque(BUFFER), - new ArrayDeque(BUFFER), new ArrayDeque(BUFFER), new ArrayDeque(BUFFER), new ArrayDeque(BUFFER), - new ArrayDeque(BUFFER), new ArrayDeque(BUFFER), new ArrayDeque(BUFFER), new ArrayDeque(BUFFER), - new ArrayDeque(BUFFER), new ArrayDeque(BUFFER), new ArrayDeque(BUFFER), new ArrayDeque(BUFFER), - new ArrayDeque(BUFFER), }; public AsyncIteratorFetcher(Iterator iterator) { this.iterator = iterator; @@ -34,41 +25,16 @@ public AsyncIteratorFetcher(Iterator iterator) { */ @Override public E get() { - - int index = (int) (Thread.currentThread().getId() % queue.length); - - // With this approach there is some risk that a queue is filled but - // never emptied. Maybe we should look for another queue to read from - // before filling our own queue? - synchronized (queue[index]) { - E poll = queue[index].poll(); - - if (poll != null) { - return poll; - } - - synchronized (this) { - poll = queue[index].poll(); - if (poll == null) { - if (iterator.hasNext()) { - poll = iterator.next(); - } - ArrayList objects = new ArrayList<>(BUFFER); - - for (int i = 0; i < BUFFER && iterator.hasNext(); i++) { - objects.add(iterator.next()); - } - - queue[index].addAll(objects); - } - - if (poll == null) { - end = true; - } - return poll; + lock.lock(); + try { + if (iterator.hasNext()) { + return iterator.next(); } + end = true; + return null; + } finally { + lock.unlock(); } - } /** diff --git a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/iterator/utils/AsyncIteratorFetcherUnordered.java b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/iterator/utils/AsyncIteratorFetcherUnordered.java new file mode 100644 index 000000000..c455e16ff --- /dev/null +++ b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/iterator/utils/AsyncIteratorFetcherUnordered.java @@ -0,0 +1,81 @@ +package com.the_qa_company.qendpoint.core.iterator.utils; + +import java.util.ArrayDeque; +import java.util.ArrayList; +import java.util.Iterator; +import java.util.Queue; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReentrantLock; +import java.util.function.Supplier; + +/** + * Synchronise an iterator + * + * @param iterator type + * @author Antoine Willerval + */ +public class AsyncIteratorFetcherUnordered extends AsyncIteratorFetcher { + public static final int BUFFER = 1024 * 4; + private final Iterator iterator; + private final Lock lock = new ReentrantLock(); + private boolean end; + Queue[] queue = new Queue[] { new ArrayDeque(BUFFER), new ArrayDeque(BUFFER), new ArrayDeque(BUFFER), + new ArrayDeque(BUFFER), new ArrayDeque(BUFFER), new ArrayDeque(BUFFER), new ArrayDeque(BUFFER), + new ArrayDeque(BUFFER), new ArrayDeque(BUFFER), new ArrayDeque(BUFFER), new ArrayDeque(BUFFER), + new ArrayDeque(BUFFER), new ArrayDeque(BUFFER), new ArrayDeque(BUFFER), new ArrayDeque(BUFFER), + new ArrayDeque(BUFFER), }; + + public AsyncIteratorFetcherUnordered(Iterator iterator) { + super(iterator); + this.iterator = iterator; + } + + /** + * @return an element from the iterator, this method is thread safe + */ + @Override + public E get() { + + int index = (int) (Thread.currentThread().getId() % queue.length); + + // With this approach there is some risk that a queue is filled but + // never emptied. Maybe we should look for another queue to read from + // before filling our own queue? + synchronized (queue[index]) { + E poll = queue[index].poll(); + + if (poll != null) { + return poll; + } + + synchronized (this) { + poll = queue[index].poll(); + if (poll == null) { + if (iterator.hasNext()) { + poll = iterator.next(); + } + ArrayList objects = new ArrayList<>(BUFFER); + + for (int i = 0; i < BUFFER && iterator.hasNext(); i++) { + objects.add(iterator.next()); + } + + queue[index].addAll(objects); + } + + if (poll == null) { + end = true; + } + return poll; + } + } + + } + + /** + * @return is the end + */ + public boolean isEnd() { + return end; + } +} diff --git a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/rdf/parsers/RDFParserRIOT.java b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/rdf/parsers/RDFParserRIOT.java index 450c8ad39..19214a3f0 100644 --- a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/rdf/parsers/RDFParserRIOT.java +++ b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/rdf/parsers/RDFParserRIOT.java @@ -79,7 +79,10 @@ private void parse(InputStream stream, String baseUri, Lang lang, boolean keepBN threads.forEach(Thread::start); for (Thread thread : threads) { try { - thread.join(); + while (thread.isAlive()) { + thread.join(1000); + } + } catch (InterruptedException e) { throw new RuntimeException(e); } diff --git a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/util/listener/MultiThreadListenerConsole.java b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/util/listener/MultiThreadListenerConsole.java index 1b28f415f..fbff6b8c7 100644 --- a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/util/listener/MultiThreadListenerConsole.java +++ b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/util/listener/MultiThreadListenerConsole.java @@ -163,8 +163,8 @@ public void removeLast() { } { - ExecutorService executorService = Executors.newSingleThreadExecutor(); - Executors.newSingleThreadExecutor().submit(() -> { + + Thread thread = new Thread(() -> { while (true) { try { Thread.sleep(500); @@ -172,10 +172,13 @@ public void removeLast() { } catch (InterruptedException e) { break; } - executorService.shutdown(); } }); + thread.setDaemon(true); + thread.setName("MultiThreadListenerConsole"); + thread.start(); + } private void render() { diff --git a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/util/string/ByteString.java b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/util/string/ByteString.java index 63b5561a8..50912763d 100644 --- a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/util/string/ByteString.java +++ b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/util/string/ByteString.java @@ -1,5 +1,7 @@ package com.the_qa_company.qendpoint.core.util.string; +import java.util.Arrays; + /** * ByteString char sequence, can't be compared with string, faster than string * with IO @@ -48,16 +50,21 @@ static ByteString copy(CharSequence csq) { */ @Override default int compareTo(ByteString other) { - int n = Math.min(length(), other.length()); - int k = 0; - while (k < n) { - char c1 = charAt(k); - char c2 = other.charAt(k); - if (c1 != c2) { - return c1 - c2; - } - k++; + + int compare = Arrays.compare(getBuffer(), other.getBuffer()); + if (compare != 0) { + return compare; } +// int n = Math.min(length(), other.length()); +// int k = 0; +// while (k < n) { +// char c1 = charAt(k); +// char c2 = other.charAt(k); +// if (c1 != c2) { +// return c1 - c2; +// } +// k++; +// } return length() - other.length(); } @@ -158,4 +165,5 @@ default boolean startsWith(ByteString prefix, int start) { } return true; } + } diff --git a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/util/string/ByteStringUtil.java b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/util/string/ByteStringUtil.java index 4526532eb..55b9ab0c9 100644 --- a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/util/string/ByteStringUtil.java +++ b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/util/string/ByteStringUtil.java @@ -23,6 +23,7 @@ import java.io.OutputStream; import java.nio.ByteBuffer; import java.nio.charset.Charset; +import java.util.Arrays; import com.the_qa_company.qendpoint.core.exceptions.NotImplementedException; import com.the_qa_company.qendpoint.core.util.io.BigByteBuffer; @@ -120,11 +121,25 @@ public static int longestCommonPrefix(CharSequence str1, CharSequence str2) { } public static int longestCommonPrefix(CharSequence str1, CharSequence str2, int from) { + + if (str1 instanceof ByteString && str2 instanceof ByteString) { + byte[] buffer = ((ByteString) str1).getBuffer(); + byte[] buffer2 = ((ByteString) str2).getBuffer(); + // System.out.println("mismatch: " + i); + int missmatch = Arrays.mismatch(buffer, from, buffer.length, buffer2, from, buffer2.length); + if (missmatch == -1) { + return Math.min(buffer.length, buffer2.length) - from; + } + return missmatch - from; + + } + int len = Math.min(str1.length(), str2.length()); int delta = from; while (delta < len && str1.charAt(delta) == str2.charAt(delta)) { delta++; } + // System.out.println("i: " + i); return delta - from; } From 1f61f151ab8becc1092224b50fbb65c1df23a645 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ha=CC=8Avard=20Ottestad?= Date: Wed, 12 Feb 2025 12:38:48 +0100 Subject: [PATCH 09/28] wip --- qendpoint-backend/pom.xml | 8 + qendpoint-core/pom.xml | 8 + .../compact/bitmap/MultiRoaringBitmap.java | 8 +- .../compact/sequence/SequenceLog64Map.java | 5 +- .../impl/section/PFCDictionarySectionBig.java | 6 +- .../impl/section/PFCDictionarySectionMap.java | 3 +- .../qendpoint/core/hdt/HDTManagerImpl.java | 4 +- .../core/hdt/impl/HDTDiskImporter.java | 2 +- .../qendpoint/core/hdt/impl/HDTImpl.java | 28 +- .../qendpoint/core/hdt/impl/WriteHDTImpl.java | 3 +- .../core/hdt/writer/TripleWriterHDT.java | 8 +- .../core/hdt/writer/TripleWriterNtriples.java | 4 +- .../iterator/utils/AsyncIteratorFetcher.java | 45 +- .../qendpoint/core/options/ControlInfo.java | 4 +- .../core/options/HDTOptionsFile.java | 8 +- .../core/storage/QEPMapIdSorter.java | 33 +- .../qendpoint/core/storage/TempBuffIn.java | 571 ++++++++++++++++++ .../qendpoint/core/storage/TempBuffOut.java | 170 ++++++ .../qendpoint/core/triples/IndexedNode.java | 1 + .../core/triples/impl/BitmapTriplesCat.java | 4 +- .../triples/impl/BitmapTriplesIndexFile.java | 7 +- .../util/LargeFakeDataSetStreamSupplier.java | 3 +- .../qendpoint/core/util/Profiler.java | 6 +- .../qendpoint/core/util/RDFInfo.java | 3 +- .../core/util/io/CloseSuppressPath.java | 7 +- .../qendpoint/core/util/io/IOUtil.java | 7 +- .../listener/MultiThreadListenerConsole.java | 11 +- .../core/util/nsd/NamespaceData.java | 6 +- .../core/util/string/ByteString.java | 50 +- .../core/util/string/ByteStringUtil.java | 44 +- .../bitmap/MultiRoaringBitmapTest.java | 8 +- .../dictionary/impl/kcat/KCatMergerTest.java | 10 +- .../section/WriteDictionarySectionTest.java | 5 +- .../qendpoint/core/rdf/parsers/TarTest.java | 7 +- .../triples/impl/BitmapQuadTriplesTest.java | 8 +- .../LargeFakeDataSetStreamSupplierTest.java | 5 +- qendpoint-store/pom.xml | 8 + .../qendpoint/compiler/SailCompiler.java | 3 +- .../qendpoint/store/EndpointStore.java | 3 +- .../qendpoint/store/MergeRunnable.java | 3 +- .../qendpoint/tools/QEPSearch.java | 3 +- .../qendpoint/utils/BitArrayDisk.java | 4 +- .../qendpoint/utils/FormatUtils.java | 3 +- .../qendpoint/store/MergeRestartTest.java | 13 +- .../qendpoint/store/Utility.java | 3 +- 45 files changed, 1035 insertions(+), 118 deletions(-) create mode 100644 qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/storage/TempBuffIn.java create mode 100644 qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/storage/TempBuffOut.java diff --git a/qendpoint-backend/pom.xml b/qendpoint-backend/pom.xml index 514353b79..8cac3f652 100644 --- a/qendpoint-backend/pom.xml +++ b/qendpoint-backend/pom.xml @@ -146,6 +146,14 @@ ${java.source.version} ${java.target.version} + + --add-exports + java.base/jdk.internal.misc=ALL-UNNAMED + --add-exports + java.base/jdk.internal.util=ALL-UNNAMED + --add-modules + jdk.incubator.vector + diff --git a/qendpoint-core/pom.xml b/qendpoint-core/pom.xml index f0df8216e..19fc512ab 100644 --- a/qendpoint-core/pom.xml +++ b/qendpoint-core/pom.xml @@ -19,6 +19,14 @@ 17 17 + + --add-exports + java.base/jdk.internal.misc=ALL-UNNAMED + --add-exports + java.base/jdk.internal.util=ALL-UNNAMED + --add-modules + jdk.incubator.vector + diff --git a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/compact/bitmap/MultiRoaringBitmap.java b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/compact/bitmap/MultiRoaringBitmap.java index d63415a78..467fc7b5f 100644 --- a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/compact/bitmap/MultiRoaringBitmap.java +++ b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/compact/bitmap/MultiRoaringBitmap.java @@ -3,6 +3,8 @@ import com.the_qa_company.qendpoint.core.exceptions.NotImplementedException; import com.the_qa_company.qendpoint.core.hdt.HDTVocabulary; import com.the_qa_company.qendpoint.core.listener.ProgressListener; +import com.the_qa_company.qendpoint.core.storage.TempBuffIn; +import com.the_qa_company.qendpoint.core.storage.TempBuffOut; import com.the_qa_company.qendpoint.core.util.io.CloseMappedByteBuffer; import com.the_qa_company.qendpoint.core.util.io.Closer; import com.the_qa_company.qendpoint.core.util.io.IOUtil; @@ -67,7 +69,7 @@ public static MultiRoaringBitmap load(InputStream input) throws IOException { * @throws IOException io exception when loading */ public static MultiRoaringBitmap load(Path input) throws IOException { - try (InputStream stream = new BufferedInputStream(Files.newInputStream(input))) { + try (InputStream stream = new TempBuffIn(Files.newInputStream(input))) { return load(stream); } } @@ -339,7 +341,7 @@ private void closeStreamBitmap(int layer, int index) throws IOException { int sizeInBytes = handle.serializedSizeInBytes(); outputMax += sizeInBytes + 8 + 8 + 1; - OutputStream os = new BufferedOutputStream(Channels.newOutputStream(output.position(loc))); + OutputStream os = new TempBuffOut(Channels.newOutputStream(output.position(loc))); os.write(BLOCK_BITMAP); IOUtil.writeLong(os, sizeInBytes); IOUtil.writeLong(os, layer); @@ -354,7 +356,7 @@ private void closeStreamBitmap(int layer, int index) throws IOException { } public void save(Path output) throws IOException { - try (OutputStream stream = new BufferedOutputStream(Files.newOutputStream(output))) { + try (OutputStream stream = new TempBuffOut(Files.newOutputStream(output))) { save(stream); } } diff --git a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/compact/sequence/SequenceLog64Map.java b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/compact/sequence/SequenceLog64Map.java index e3ce9942d..809507378 100644 --- a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/compact/sequence/SequenceLog64Map.java +++ b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/compact/sequence/SequenceLog64Map.java @@ -25,6 +25,7 @@ import com.the_qa_company.qendpoint.core.exceptions.NotImplementedException; import com.the_qa_company.qendpoint.core.hdt.HDTVocabulary; import com.the_qa_company.qendpoint.core.listener.ProgressListener; +import com.the_qa_company.qendpoint.core.storage.TempBuffIn; import com.the_qa_company.qendpoint.core.util.BitUtil; import com.the_qa_company.qendpoint.core.util.crc.CRC32; import com.the_qa_company.qendpoint.core.util.crc.CRC8; @@ -65,7 +66,7 @@ public class SequenceLog64Map implements Sequence, Closeable { public SequenceLog64Map(File f) throws IOException { // Read from the beginning of the file - this(new CountInputStream(new BufferedInputStream(new FileInputStream(f))), f, true); + this(new CountInputStream(new TempBuffIn(new FileInputStream(f))), f, true); } public SequenceLog64Map(CountInputStream in, File f) throws IOException { @@ -162,7 +163,7 @@ private void mapFiles(File f, long base) throws IOException { // FIXME: Bug in the previous code, find what because it should be more // efficient - CountInputStream in = new CountInputStream(new BufferedInputStream(new FileInputStream(f))); + CountInputStream in = new CountInputStream(new TempBuffIn(new FileInputStream(f))); IOUtil.skip(in, base + ((numwords - 1) * 8L)); // System.out.println("Last word starts at: "+in.getTotalBytes()); // Read only used bits from last entry (byte aligned, little endian) diff --git a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/dictionary/impl/section/PFCDictionarySectionBig.java b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/dictionary/impl/section/PFCDictionarySectionBig.java index 9a8a0fb54..7dd61e438 100644 --- a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/dictionary/impl/section/PFCDictionarySectionBig.java +++ b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/dictionary/impl/section/PFCDictionarySectionBig.java @@ -39,6 +39,8 @@ import com.the_qa_company.qendpoint.core.exceptions.IllegalFormatException; import com.the_qa_company.qendpoint.core.listener.ProgressListener; import com.the_qa_company.qendpoint.core.options.HDTOptions; +import com.the_qa_company.qendpoint.core.storage.TempBuffIn; +import com.the_qa_company.qendpoint.core.storage.TempBuffOut; import com.the_qa_company.qendpoint.core.util.BitUtil; import com.the_qa_company.qendpoint.core.util.Mutable; import com.the_qa_company.qendpoint.core.util.crc.CRC32; @@ -116,7 +118,7 @@ public void load(Iterator it, long numentries, ProgressL ByteString previousStr = null; try { - try (OutputStream out = new BufferedOutputStream(new FileOutputStream(file))) { + try (OutputStream out = new TempBuffOut(new FileOutputStream(file))) { while (it.hasNext()) { ByteString str = ByteString.of(it.next()); @@ -161,7 +163,7 @@ public void load(Iterator it, long numentries, ProgressL byteOut.writeTo(out); } - try (InputStream in = new BufferedInputStream(new FileInputStream(file))) { + try (InputStream in = new TempBuffIn(new FileInputStream(file))) { // Read block by block // Read packed data diff --git a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/dictionary/impl/section/PFCDictionarySectionMap.java b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/dictionary/impl/section/PFCDictionarySectionMap.java index 1a0775aac..59410ea50 100644 --- a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/dictionary/impl/section/PFCDictionarySectionMap.java +++ b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/dictionary/impl/section/PFCDictionarySectionMap.java @@ -25,6 +25,7 @@ import com.the_qa_company.qendpoint.core.exceptions.IllegalFormatException; import com.the_qa_company.qendpoint.core.exceptions.NotImplementedException; import com.the_qa_company.qendpoint.core.listener.ProgressListener; +import com.the_qa_company.qendpoint.core.storage.TempBuffIn; import com.the_qa_company.qendpoint.core.util.io.BigMappedByteBuffer; import com.the_qa_company.qendpoint.core.compact.integer.VByte; import com.the_qa_company.qendpoint.core.compact.sequence.Sequence; @@ -375,7 +376,7 @@ public void load(Iterator it, long count, ProgressListen @Override public void save(OutputStream output, ProgressListener listener) throws IOException { - InputStream in = new BufferedInputStream(new FileInputStream(f)); + InputStream in = new TempBuffIn(new FileInputStream(f)); IOUtil.skip(in, startOffset); IOUtil.copyStream(in, output, endOffset - startOffset); in.close(); diff --git a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/hdt/HDTManagerImpl.java b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/hdt/HDTManagerImpl.java index 492421d91..b9638935e 100644 --- a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/hdt/HDTManagerImpl.java +++ b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/hdt/HDTManagerImpl.java @@ -20,6 +20,7 @@ import com.the_qa_company.qendpoint.core.rdf.RDFParserCallback; import com.the_qa_company.qendpoint.core.rdf.RDFParserFactory; import com.the_qa_company.qendpoint.core.rdf.TripleWriter; +import com.the_qa_company.qendpoint.core.storage.TempBuffOut; import com.the_qa_company.qendpoint.core.triples.TripleString; import com.the_qa_company.qendpoint.core.util.BitUtil; import com.the_qa_company.qendpoint.core.util.Profiler; @@ -254,8 +255,7 @@ public HDTResult doGenerateHDT(String rdfFileName, String baseURI, RDFNotation r InputStream stream = readIs.is(); try (InputStream is = checksumPath != null ? new CRCInputStream(stream, new CRC32()) : stream; - OutputStream os = new BufferedOutputStream( - Files.newOutputStream(preDownload, openOptions))) { + OutputStream os = new TempBuffOut(Files.newOutputStream(preDownload, openOptions))) { IOUtil.copy(is, os, listener, 10_000_000); if (is instanceof CRCInputStream crcIs) { checksum = crcIs.getCRC().getValue(); diff --git a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/hdt/impl/HDTDiskImporter.java b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/hdt/impl/HDTDiskImporter.java index fe03550bd..384236ab2 100644 --- a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/hdt/impl/HDTDiskImporter.java +++ b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/hdt/impl/HDTDiskImporter.java @@ -187,7 +187,7 @@ public CompressTripleMapper compressDictionary(Iterator iterator) + "iB with " + ways + "ways and " + workers + " worker(s)"); AsyncIteratorFetcherUnordered source = new AsyncIteratorFetcherUnordered<>(iterator); - +// AsyncIteratorFetcher source = new AsyncIteratorFetcher<>(iterator); profiler.pushSection("section compression"); CompressionResult compressionResult; try { diff --git a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/hdt/impl/HDTImpl.java b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/hdt/impl/HDTImpl.java index 868e63db4..8ee2af5e7 100644 --- a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/hdt/impl/HDTImpl.java +++ b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/hdt/impl/HDTImpl.java @@ -52,6 +52,8 @@ import com.the_qa_company.qendpoint.core.options.HDTOptions; import com.the_qa_company.qendpoint.core.options.HDTOptionsKeys; import com.the_qa_company.qendpoint.core.options.HDTSpecification; +import com.the_qa_company.qendpoint.core.storage.TempBuffIn; +import com.the_qa_company.qendpoint.core.storage.TempBuffOut; import com.the_qa_company.qendpoint.core.triples.DictionaryEntriesDiff; import com.the_qa_company.qendpoint.core.triples.IteratorTripleID; import com.the_qa_company.qendpoint.core.triples.IteratorTripleString; @@ -160,9 +162,9 @@ public void loadFromHDT(InputStream input, ProgressListener listener) throws IOE public void loadFromHDT(String hdtFileName, ProgressListener listener) throws IOException { InputStream in; if (hdtFileName.endsWith(".gz")) { - in = new BufferedInputStream(new GZIPInputStream(new FileInputStream(hdtFileName))); + in = new TempBuffIn(new GZIPInputStream(new FileInputStream(hdtFileName))); } else { - in = new CountInputStream(new BufferedInputStream(new FileInputStream(hdtFileName))); + in = new CountInputStream(new TempBuffIn(new FileInputStream(hdtFileName))); } loadFromHDT(in, listener); in.close(); @@ -192,7 +194,7 @@ public void mapFromHDT(File f, long offset, ProgressListener listener) throws IO } boolean dumpBinInfo = spec.getBoolean(HDTOptionsKeys.DUMP_BINARY_OFFSETS, false); - try (CountInputStream input = new CountInputStream(new BufferedInputStream(new FileInputStream(hdtFileName)), + try (CountInputStream input = new CountInputStream(new TempBuffIn(new FileInputStream(hdtFileName)), dumpBinInfo)) { input.printIndex("HDT CI"); @@ -256,7 +258,7 @@ public void mapFromHDT(File f, long offset, ProgressListener listener) throws IO */ @Override public void saveToHDT(String fileName, ProgressListener listener) throws IOException { - try (OutputStream out = new BufferedOutputStream(new FileOutputStream(fileName))) { + try (OutputStream out = new TempBuffOut(new FileOutputStream(fileName))) { // OutputStream out = new GZIPOutputStream(new // BufferedOutputStream(new FileOutputStream(fileName))); saveToHDT(out, listener); @@ -510,7 +512,7 @@ public void loadOrCreateIndex(ProgressListener listener, HDTOptions spec) throws } CountInputStream in = null; try { - in = new CountInputStream(new BufferedInputStream(new FileInputStream(ff))); + in = new CountInputStream(new TempBuffIn(new FileInputStream(ff))); ci.load(in); if (isMapped) { triples.mapIndex(in, new File(indexName), ci, listener); @@ -528,9 +530,9 @@ public void loadOrCreateIndex(ProgressListener listener, HDTOptions spec) throws // SAVE if (this.hdtFileName != null) { - BufferedOutputStream out = null; + OutputStream out = null; try { - out = new BufferedOutputStream(new FileOutputStream(versionName)); + out = new TempBuffOut(new FileOutputStream(versionName)); ci.clear(); triples.saveIndex(out, ci, listener); out.close(); @@ -611,7 +613,7 @@ public void cat(String location, HDT hdt1, HDT hdt2, ProgressListener listener, // map the generated dictionary FourSectionDictionaryBig dictionary; try (CountInputStream fis = new CountInputStream( - new BufferedInputStream(new FileInputStream(location + "dictionary")))) { + new TempBuffIn(new FileInputStream(location + "dictionary")))) { dictionary = new FourSectionDictionaryBig(new HDTSpecification()); fis.mark(1024); ci2.load(fis); @@ -655,8 +657,7 @@ public void cat(String location, HDT hdt1, HDT hdt2, ProgressListener listener, Files.delete(Paths.get(location + "O2" + "Types")); // map the triples - try (CountInputStream fis2 = new CountInputStream( - new BufferedInputStream(new FileInputStream(location + "triples")))) { + try (CountInputStream fis2 = new CountInputStream(new TempBuffIn(new FileInputStream(location + "triples")))) { ControlInfo ci2 = new ControlInformation(); ci2.clear(); fis2.mark(1024); @@ -695,7 +696,7 @@ public void catCustom(String location, HDT hdt1, HDT hdt2, ProgressListener list // map the generated dictionary ControlInfo ci2 = new ControlInformation(); try (CountInputStream fis = new CountInputStream( - new BufferedInputStream(new FileInputStream(location + "dictionary")))) { + new TempBuffIn(new FileInputStream(location + "dictionary")))) { HDTSpecification spec = new HDTSpecification(); spec.set(HDTOptionsKeys.TEMP_DICTIONARY_IMPL_KEY, HDTOptionsKeys.TEMP_DICTIONARY_IMPL_VALUE_MULT_HASH); spec.set(HDTOptionsKeys.DICTIONARY_TYPE_KEY, HDTOptionsKeys.DICTIONARY_TYPE_VALUE_MULTI_OBJECTS); @@ -757,8 +758,7 @@ public void catCustom(String location, HDT hdt1, HDT hdt2, ProgressListener list Files.delete(Paths.get(location + "O2")); Files.delete(Paths.get(location + "O2" + "Types")); // map the triples - try (CountInputStream fis2 = new CountInputStream( - new BufferedInputStream(new FileInputStream(location + "triples")))) { + try (CountInputStream fis2 = new CountInputStream(new TempBuffIn(new FileInputStream(location + "triples")))) { ControlInformation ci2 = new ControlInformation(); ci2.clear(); fis2.mark(1024); @@ -815,7 +815,7 @@ public void diffBit(String location, HDT hdt, Bitmap deleteBitmap, ProgressListe ControlInfo ci2 = new ControlInformation(); try (CountInputStream fis = new CountInputStream( - new BufferedInputStream(new FileInputStream(location + "dictionary")))) { + new TempBuffIn(new FileInputStream(location + "dictionary")))) { fis.mark(1024); ci2.load(fis); fis.reset(); diff --git a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/hdt/impl/WriteHDTImpl.java b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/hdt/impl/WriteHDTImpl.java index e9f6cf28e..dbccae45e 100644 --- a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/hdt/impl/WriteHDTImpl.java +++ b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/hdt/impl/WriteHDTImpl.java @@ -8,6 +8,7 @@ import com.the_qa_company.qendpoint.core.header.HeaderPrivate; import com.the_qa_company.qendpoint.core.listener.ProgressListener; import com.the_qa_company.qendpoint.core.options.HDTOptions; +import com.the_qa_company.qendpoint.core.storage.TempBuffOut; import com.the_qa_company.qendpoint.core.triples.IteratorTripleString; import com.the_qa_company.qendpoint.core.triples.TriplesPrivate; import com.the_qa_company.qendpoint.core.triples.impl.WriteBitmapTriples; @@ -87,7 +88,7 @@ public void loadOrCreateIndex(ProgressListener listener, HDTOptions disk) { @Override public void saveToHDT(String fileName, ProgressListener listener) throws IOException { - try (OutputStream out = new BufferedOutputStream(Files.newOutputStream(Path.of(fileName)))) { + try (OutputStream out = new TempBuffOut(Files.newOutputStream(Path.of(fileName)))) { saveToHDT(out, listener); } } diff --git a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/hdt/writer/TripleWriterHDT.java b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/hdt/writer/TripleWriterHDT.java index b8260f78b..27c1443a5 100644 --- a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/hdt/writer/TripleWriterHDT.java +++ b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/hdt/writer/TripleWriterHDT.java @@ -18,6 +18,7 @@ import com.the_qa_company.qendpoint.core.listener.ProgressListener; import com.the_qa_company.qendpoint.core.options.HDTOptions; import com.the_qa_company.qendpoint.core.rdf.TripleWriter; +import com.the_qa_company.qendpoint.core.storage.TempBuffOut; import com.the_qa_company.qendpoint.core.triples.TempTriples; import com.the_qa_company.qendpoint.core.triples.TripleString; import com.the_qa_company.qendpoint.core.util.StopWatch; @@ -39,10 +40,9 @@ public TripleWriterHDT(String baseUri, HDTOptions spec, String outFile, boolean this.baseUri = baseUri; this.spec = spec; if (compress) { - this.out = new BufferedOutputStream( - new GZIPOutputStream(new BufferedOutputStream(new FileOutputStream(outFile)))); + this.out = new TempBuffOut(new GZIPOutputStream(new TempBuffOut(new FileOutputStream(outFile)))); } else { - this.out = new BufferedOutputStream(new FileOutputStream(outFile)); + this.out = new TempBuffOut(new FileOutputStream(outFile)); } close = true; init(); @@ -51,7 +51,7 @@ public TripleWriterHDT(String baseUri, HDTOptions spec, String outFile, boolean public TripleWriterHDT(String baseUri, HDTOptions spec, OutputStream out) { this.baseUri = baseUri; this.spec = spec; - this.out = new BufferedOutputStream(out); + this.out = new TempBuffOut(out); init(); } diff --git a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/hdt/writer/TripleWriterNtriples.java b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/hdt/writer/TripleWriterNtriples.java index f7d0ea669..c0934124b 100644 --- a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/hdt/writer/TripleWriterNtriples.java +++ b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/hdt/writer/TripleWriterNtriples.java @@ -11,6 +11,7 @@ import java.util.zip.GZIPOutputStream; import com.the_qa_company.qendpoint.core.rdf.TripleWriter; +import com.the_qa_company.qendpoint.core.storage.TempBuffOut; import com.the_qa_company.qendpoint.core.triples.TripleString; public class TripleWriterNtriples implements TripleWriter { @@ -20,8 +21,7 @@ public class TripleWriterNtriples implements TripleWriter { public TripleWriterNtriples(String outFile, boolean compress) throws IOException { if (compress) { - this.out = new OutputStreamWriter( - new GZIPOutputStream(new BufferedOutputStream(new FileOutputStream(outFile)))); + this.out = new OutputStreamWriter(new GZIPOutputStream(new TempBuffOut(new FileOutputStream(outFile)))); } else { this.out = new BufferedWriter(new FileWriter(outFile)); } diff --git a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/iterator/utils/AsyncIteratorFetcher.java b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/iterator/utils/AsyncIteratorFetcher.java index 788b3ab8a..d45362264 100644 --- a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/iterator/utils/AsyncIteratorFetcher.java +++ b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/iterator/utils/AsyncIteratorFetcher.java @@ -1,6 +1,8 @@ package com.the_qa_company.qendpoint.core.iterator.utils; +import java.util.ArrayList; import java.util.Iterator; +import java.util.concurrent.ConcurrentLinkedQueue; import java.util.concurrent.locks.Lock; import java.util.concurrent.locks.ReentrantLock; import java.util.function.Supplier; @@ -16,6 +18,8 @@ public class AsyncIteratorFetcher implements Supplier { private final Lock lock = new ReentrantLock(); private boolean end; + volatile ConcurrentLinkedQueue queue = new ConcurrentLinkedQueue<>(); + public AsyncIteratorFetcher(Iterator iterator) { this.iterator = iterator; } @@ -24,16 +28,45 @@ public AsyncIteratorFetcher(Iterator iterator) { * @return an element from the iterator, this method is thread safe */ @Override +// public E get() { +// lock.lock(); +// try { +// if (iterator.hasNext()) { +// return iterator.next(); +// } +// end = true; +// return null; +// } finally { +// lock.unlock(); +// } +// } + public E get() { - lock.lock(); - try { - if (iterator.hasNext()) { - return iterator.next(); + E poll = queue.poll(); + if (poll != null) { + return poll; + } + + synchronized (this) { + poll = queue.poll(); + if (poll != null) { + return poll; + } + + ConcurrentLinkedQueue newqueue = new ConcurrentLinkedQueue<>(); + + for (int i = 0; i < 128 && iterator.hasNext(); i++) { + if (poll == null) { + poll = iterator.next(); + } + newqueue.add(iterator.next()); + } + this.queue = newqueue; + if (poll != null) { + return poll; } end = true; return null; - } finally { - lock.unlock(); } } diff --git a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/options/ControlInfo.java b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/options/ControlInfo.java index d1c1696ef..2fb603c72 100644 --- a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/options/ControlInfo.java +++ b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/options/ControlInfo.java @@ -19,6 +19,8 @@ package com.the_qa_company.qendpoint.core.options; +import com.the_qa_company.qendpoint.core.storage.TempBuffOut; + import java.io.BufferedOutputStream; import java.io.IOException; import java.io.InputStream; @@ -43,7 +45,7 @@ enum Type { void setFormat(String format); default void save(Path filename) throws IOException { - try (BufferedOutputStream os = new BufferedOutputStream(Files.newOutputStream(filename))) { + try (OutputStream os = new TempBuffOut(Files.newOutputStream(filename))) { save(os); } } diff --git a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/options/HDTOptionsFile.java b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/options/HDTOptionsFile.java index 8f93c5f39..4c83f8ef2 100644 --- a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/options/HDTOptionsFile.java +++ b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/options/HDTOptionsFile.java @@ -2,6 +2,8 @@ import com.the_qa_company.qendpoint.core.compact.integer.VByte; import com.the_qa_company.qendpoint.core.listener.ProgressListener; +import com.the_qa_company.qendpoint.core.storage.TempBuffIn; +import com.the_qa_company.qendpoint.core.storage.TempBuffOut; import com.the_qa_company.qendpoint.core.util.crc.CRC32; import com.the_qa_company.qendpoint.core.util.crc.CRCInputStream; import com.the_qa_company.qendpoint.core.util.crc.CRCOutputStream; @@ -37,8 +39,7 @@ public void sync() throws IOException { return; } ProgressListener l = ProgressListener.ignore(); - try (CRCInputStream is = new CRCInputStream(new BufferedInputStream(Files.newInputStream(location)), - new CRC32())) { + try (CRCInputStream is = new CRCInputStream(new TempBuffIn(Files.newInputStream(location)), new CRC32())) { if (IOUtil.readLong(is) != MAGIC) throw new IOException("Can't read HDTOptions file: Bad magic"); @@ -59,8 +60,7 @@ public void sync() throws IOException { public void save() throws IOException { ProgressListener l = ProgressListener.ignore(); - try (CRCOutputStream os = new CRCOutputStream(new BufferedOutputStream(Files.newOutputStream(location)), - new CRC32())) { + try (CRCOutputStream os = new CRCOutputStream(new TempBuffOut(Files.newOutputStream(location)), new CRC32())) { IOUtil.writeLong(os, MAGIC); Set keys = options.getKeys(); VByte.encode(os, keys.size()); diff --git a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/storage/QEPMapIdSorter.java b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/storage/QEPMapIdSorter.java index c53fb7db6..3d8b4509a 100644 --- a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/storage/QEPMapIdSorter.java +++ b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/storage/QEPMapIdSorter.java @@ -20,10 +20,12 @@ import java.io.Closeable; import java.io.IOException; import java.io.InputStream; +import java.io.OutputStream; import java.nio.file.Files; import java.nio.file.Path; import java.util.ArrayList; import java.util.Arrays; +import java.util.Collections; import java.util.Iterator; import java.util.List; import java.util.function.Supplier; @@ -92,7 +94,7 @@ public void sort() throws IOException { CloseSuppressPath output = merger.waitResult().orElse(null); if (output != null) { - try (BufferedInputStream stream = new BufferedInputStream(Files.newInputStream(output))) { + try (InputStream stream = new TempBuffIn(Files.newInputStream(output))) { QEPMapReader reader = new QEPMapReader(stream); long index = 0; @@ -154,7 +156,7 @@ private record Merger(long chunkSize) implements KWayMerger.KWayMergerImpl flux, CloseSuppressPath output) throws KWayMerger.KWayMergerException { - try (BufferedOutputStream stream = new BufferedOutputStream(Files.newOutputStream(output))) { + try (OutputStream stream = new TempBuffOut(Files.newOutputStream(output))) { QEPMapIds ids; List idList = new ArrayList<>(); @@ -163,12 +165,25 @@ public void createChunk(Supplier flux, CloseSuppressPath output) idList.add(ids); } - idList.sort(QEPMapIds::compareTo); + idList.parallelStream().sorted(QEPMapIds::compareTo).forEachOrdered(qepMapIds -> { + try { + VByte.encode(stream, qepMapIds.origin()); + } catch (IOException e) { + throw new RuntimeException(e); + } + try { + VByte.encode(stream, qepMapIds.destination()); + } catch (IOException e) { + throw new RuntimeException(e); + } + }); - for (QEPMapIds qepMapIds : idList) { - VByte.encode(stream, qepMapIds.origin()); - VByte.encode(stream, qepMapIds.destination()); - } +// idList.sort(QEPMapIds::compareTo); +// +// for (QEPMapIds qepMapIds : idList) { +// VByte.encode(stream, qepMapIds.origin()); +// VByte.encode(stream, qepMapIds.destination()); +// } VByte.encode(stream, 0); VByte.encode(stream, 0); @@ -184,7 +199,7 @@ public void mergeChunks(List inputs, CloseSuppressPath output InputStream[] pathInput = new InputStream[inputs.size()]; for (int i = 0; i < pathInput.length; i++) { - pathInput[i] = new BufferedInputStream(Files.newInputStream(inputs.get(i))); + pathInput[i] = new TempBuffIn(Files.newInputStream(inputs.get(i))); } try { @@ -192,7 +207,7 @@ public void mergeChunks(List inputs, CloseSuppressPath output ExceptionIterator tree = MergeExceptionIterator .buildOfTree(QEPMapReader::new, Arrays.asList(pathInput), 0, inputs.size()); - try (BufferedOutputStream stream = new BufferedOutputStream(Files.newOutputStream(output))) { + try (OutputStream stream = new TempBuffOut(Files.newOutputStream(output))) { while (tree.hasNext()) { QEPMapIds ids = tree.next(); VByte.encode(stream, ids.origin()); diff --git a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/storage/TempBuffIn.java b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/storage/TempBuffIn.java new file mode 100644 index 000000000..1050e92bb --- /dev/null +++ b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/storage/TempBuffIn.java @@ -0,0 +1,571 @@ +/* + * Copyright (c) 1994, 2024, Oracle and/or its affiliates. All rights reserved. + * DO NOT ALTER OR REMOVE COPYRIGHT NOTICES OR THIS FILE HEADER. This code is + * free software; you can redistribute it and/or modify it under the terms of + * the GNU General Public License version 2 only, as published by the Free + * Software Foundation. Oracle designates this particular file as subject to the + * "Classpath" exception as provided by Oracle in the LICENSE file that + * accompanied this code. This code is distributed in the hope that it will be + * useful, but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the GNU General + * Public License version 2 for more details (a copy is included in the LICENSE + * file that accompanied this code). You should have received a copy of the GNU + * General Public License version 2 along with this work; if not, write to the + * Free Software Foundation, Inc., 51 Franklin St, Fifth Floor, Boston, MA + * 02110-1301 USA. Please contact Oracle, 500 Oracle Parkway, Redwood Shores, CA + * 94065 USA or visit www.oracle.com if you need additional information or have + * any questions. + */ + +package com.the_qa_company.qendpoint.core.storage; + +import jdk.internal.misc.Unsafe; +import jdk.internal.util.ArraysSupport; + +import java.io.ByteArrayOutputStream; +import java.io.FileOutputStream; +import java.io.FilterInputStream; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.io.PipedOutputStream; +import java.util.Arrays; +import java.util.Objects; + +/** + * A {@code TempBuffIn} adds functionality to another input stream-namely, the + * ability to buffer the input and to support the {@code mark} and {@code reset} + * methods. When the {@code TempBuffIn} is created, an internal buffer array is + * created. As bytes from the stream are read or skipped, the internal buffer is + * refilled as necessary from the contained input stream, many bytes at a time. + * The {@code mark} operation remembers a point in the input stream and the + * {@code reset} operation causes all the bytes read since the most recent + * {@code mark} operation to be reread before new bytes are taken from the + * contained input stream. + * + * @author Arthur van Hoff + * @since 1.0 + */ +public class TempBuffIn extends FilterInputStream { + + private static final int DEFAULT_BUFFER_SIZE = 8192; + + private static final byte[] EMPTY = new byte[0]; + + /** + * As this class is used early during bootstrap, it's motivated to use + * Unsafe.compareAndSetReference instead of AtomicReferenceFieldUpdater (or + * VarHandles) to reduce dependencies and improve startup time. + */ + private static final Unsafe U = Unsafe.getUnsafe(); + + private static final long BUF_OFFSET = U.objectFieldOffset(TempBuffIn.class, "buf"); + + // initial buffer size (DEFAULT_BUFFER_SIZE or size specified to + // constructor) + private final int initialSize; + + /** + * The internal buffer array where the data is stored. When necessary, it + * may be replaced by another array of a different size. + */ + /* + * We null this out with a CAS on close(), which is necessary since closes + * can be asynchronous. We use nullness of buf[] as primary indicator that + * this stream is closed. (The "in" field is also nulled out on close.) + */ + protected volatile byte[] buf; + + /** + * The index one greater than the index of the last valid byte in the + * buffer. This value is always in the range {@code 0} through + * {@code buf.length}; elements {@code buf[0]} through {@code buf[count-1]} + * contain buffered input data obtained from the underlying input stream. + */ + protected int count; + + /** + * The current position in the buffer. This is the index of the next byte to + * be read from the {@code buf} array. + *

+ * This value is always in the range {@code 0} through {@code count}. If it + * is less than {@code count}, then {@code buf[pos]} is the next byte to be + * supplied as input; if it is equal to {@code count}, then the next + * {@code read} or {@code skip} operation will require more bytes to be read + * from the contained input stream. + */ + protected int pos; + + /** + * The value of the {@code pos} field at the time the last {@code mark} + * method was called. + *

+ * This value is always in the range {@code -1} through {@code pos}. If + * there is no marked position in the input stream, this field is + * {@code -1}. If there is a marked position in the input stream, then + * {@code buf[markpos]} is the first byte to be supplied as input after a + * {@code reset} operation. If {@code markpos} is not {@code -1}, then all + * bytes from positions {@code buf[markpos]} through {@code buf[pos-1]} must + * remain in the buffer array (though they may be moved to another place in + * the buffer array, with suitable adjustments to the values of + * {@code count}, {@code pos}, and {@code markpos}); they may not be + * discarded unless and until the difference between {@code pos} and + * {@code markpos} exceeds {@code marklimit}. + */ + protected int markpos = -1; + + /** + * The maximum read ahead allowed after a call to the {@code mark} method + * before subsequent calls to the {@code reset} method fail. Whenever the + * difference between {@code pos} and {@code markpos} exceeds + * {@code marklimit}, then the mark may be dropped by setting + * {@code markpos} to {@code -1}. + */ + protected int marklimit; + + /** + * Check to make sure that underlying input stream has not been nulled out + * due to close; if not return it; + */ + private InputStream getInIfOpen() throws IOException { + InputStream input = in; + if (input == null) { + throw new IOException("Stream closed"); + } + return input; + } + + /** + * Returns the internal buffer, optionally allocating it if empty. + * + * @param allocateIfEmpty true to allocate if empty + * @throws IOException if the stream is closed (buf is null) + */ + private byte[] getBufIfOpen(boolean allocateIfEmpty) throws IOException { + byte[] buffer = buf; + if (allocateIfEmpty && buffer == EMPTY) { + buffer = new byte[initialSize]; + if (!U.compareAndSetReference(this, BUF_OFFSET, EMPTY, buffer)) { + // re-read buf + buffer = buf; + } + } + if (buffer == null) { + throw new IOException("Stream closed"); + } + return buffer; + } + + /** + * Returns the internal buffer, allocating it if empty. + * + * @throws IOException if the stream is closed (buf is null) + */ + private byte[] getBufIfOpen() throws IOException { + return getBufIfOpen(true); + } + + /** + * Throws IOException if the stream is closed (buf is null). + */ + private void ensureOpen() throws IOException { + if (buf == null) { + throw new IOException("Stream closed"); + } + } + + /** + * Creates a {@code TempBuffIn} and saves its argument, the input stream + * {@code in}, for later use. An internal buffer array is created and stored + * in {@code buf}. + * + * @param in the underlying input stream. + */ + public TempBuffIn(InputStream in) { + this(in, DEFAULT_BUFFER_SIZE); + } + + /** + * Creates a {@code TempBuffIn} with the specified buffer size, and saves + * its argument, the input stream {@code in}, for later use. An internal + * buffer array of length {@code size} is created and stored in {@code buf}. + * + * @param in the underlying input stream. + * @param size the buffer size. + * @throws IllegalArgumentException if {@code size <= 0}. + */ + public TempBuffIn(InputStream in, int size) { + super(in); + if (size <= 0) { + throw new IllegalArgumentException("Buffer size <= 0"); + } + initialSize = size; + + buf = new byte[size]; + + } + + /** + * Fills the buffer with more data, taking into account shuffling and other + * tricks for dealing with marks. Assumes that it is being called by a + * locked method. This method also assumes that all data has already been + * read in, hence pos > count. + */ + private void fill() throws IOException { + byte[] buffer = getBufIfOpen(); + if (markpos == -1) { + pos = 0; /* no mark: throw away the buffer */ + } else if (pos >= buffer.length) { /* no room left in buffer */ + if (markpos > 0) { /* can throw away early part of the buffer */ + int sz = pos - markpos; + System.arraycopy(buffer, markpos, buffer, 0, sz); + pos = sz; + markpos = 0; + } else if (buffer.length >= marklimit) { + markpos = -1; /* buffer got too big, invalidate mark */ + pos = 0; /* drop buffer contents */ + } else { /* grow buffer */ + int nsz = ArraysSupport.newLength(pos, 1, /* minimum growth */ + pos /* preferred growth */); + if (nsz > marklimit) { + nsz = marklimit; + } + byte[] nbuf = new byte[nsz]; + System.arraycopy(buffer, 0, nbuf, 0, pos); + if (!U.compareAndSetReference(this, BUF_OFFSET, buffer, nbuf)) { + // Can't replace buf if there was an async close. + // Note: This would need to be changed if fill() + // is ever made accessible to multiple threads. + // But for now, the only way CAS can fail is via close. + // assert buf == null; + throw new IOException("Stream closed"); + } + buffer = nbuf; + } + } + count = pos; + int n = getInIfOpen().read(buffer, pos, buffer.length - pos); + if (n > 0) { + count = n + pos; + } + } + + /** + * See the general contract of the {@code read} method of + * {@code InputStream}. + * + * @return the next byte of data, or {@code -1} if the end of the stream is + * reached. + * @throws IOException if this input stream has been closed by invoking its + * {@link #close()} method, or an I/O error occurs. + */ + public int read() throws IOException { + + return implRead(); + + } + + private int implRead() throws IOException { + if (pos >= count) { + fill(); + if (pos >= count) { + return -1; + } + } + return getBufIfOpen()[pos++] & 0xff; + } + + /** + * Read bytes into a portion of an array, reading from the underlying stream + * at most once if necessary. + */ + private int read1(byte[] b, int off, int len) throws IOException { + int avail = count - pos; + if (avail <= 0) { + /* + * If the requested length is at least as large as the buffer, and + * if there is no mark/reset activity, do not bother to copy the + * bytes into the local buffer. In this way buffered streams will + * cascade harmlessly. + */ + int size = Math.max(getBufIfOpen(false).length, initialSize); + if (len >= size && markpos == -1) { + return getInIfOpen().read(b, off, len); + } + fill(); + avail = count - pos; + if (avail <= 0) { + return -1; + } + } + int cnt = (avail < len) ? avail : len; + System.arraycopy(getBufIfOpen(), pos, b, off, cnt); + pos += cnt; + return cnt; + } + + /** + * Reads bytes from this byte-input stream into the specified byte array, + * starting at the given offset. + *

+ * This method implements the general contract of the corresponding + * {@link InputStream#read(byte[], int, int) read} method of the + * {@link InputStream} class. As an additional convenience, it attempts to + * read as many bytes as possible by repeatedly invoking the {@code read} + * method of the underlying stream. This iterated {@code read} continues + * until one of the following conditions becomes true: + *

    + *
  • The specified number of bytes have been read, + *
  • The {@code read} method of the underlying stream returns {@code -1}, + * indicating end-of-file, or + *
  • The {@code available} method of the underlying stream returns zero, + * indicating that further input requests would block. + *
+ * If the first {@code read} on the underlying stream returns {@code -1} to + * indicate end-of-file then this method returns {@code -1}. Otherwise, this + * method returns the number of bytes actually read. + *

+ * Subclasses of this class are encouraged, but not required, to attempt to + * read as many bytes as possible in the same fashion. + * + * @param b destination buffer. + * @param off offset at which to start storing bytes. + * @param len maximum number of bytes to read. + * @return the number of bytes read, or {@code -1} if the end of the stream + * has been reached. + * @throws IOException if this input stream has been closed by + * invoking its {@link #close()} method, + * or an I/O error occurs. + * @throws IndexOutOfBoundsException {@inheritDoc} + */ + public int read(byte[] b, int off, int len) throws IOException { + + return implRead(b, off, len); + + } + + private int implRead(byte[] b, int off, int len) throws IOException { + ensureOpen(); + if ((off | len | (off + len) | (b.length - (off + len))) < 0) { + throw new IndexOutOfBoundsException(); + } else if (len == 0) { + return 0; + } + + int n = 0; + for (;;) { + int nread = read1(b, off + n, len - n); + if (nread <= 0) { + return (n == 0) ? nread : n; + } + n += nread; + if (n >= len) { + return n; + } + // if not closed but no bytes available, return + InputStream input = in; + if (input != null && input.available() <= 0) { + return n; + } + } + } + + /** + * See the general contract of the {@code skip} method of + * {@code InputStream}. + * + * @throws IOException if this input stream has been closed by invoking its + * {@link #close()} method, {@code in.skip(n)} throws an + * IOException, or an I/O error occurs. + */ + public long skip(long n) throws IOException { + + return implSkip(n); + + } + + private long implSkip(long n) throws IOException { + ensureOpen(); + if (n <= 0) { + return 0; + } + long avail = count - pos; + + if (avail <= 0) { + // If no mark position set then don't keep in buffer + if (markpos == -1) { + return getInIfOpen().skip(n); + } + + // Fill in buffer to save bytes for reset + fill(); + avail = count - pos; + if (avail <= 0) { + return 0; + } + } + + long skipped = (avail < n) ? avail : n; + pos += (int) skipped; + return skipped; + } + + /** + * Returns an estimate of the number of bytes that can be read (or skipped + * over) from this input stream without blocking by the next invocation of a + * method for this input stream. The next invocation might be the same + * thread or another thread. A single read or skip of this many bytes will + * not block, but may read or skip fewer bytes. + *

+ * This method returns the sum of the number of bytes remaining to be read + * in the buffer ({@code count - pos}) and the result of calling the + * {@link java.io.FilterInputStream#in in}{@code .available()}. + * + * @return an estimate of the number of bytes that can be read (or skipped + * over) from this input stream without blocking. + * @throws IOException if this input stream has been closed by invoking its + * {@link #close()} method, or an I/O error occurs. + */ + public int available() throws IOException { + + return implAvailable(); + + } + + private int implAvailable() throws IOException { + int n = count - pos; + int avail = getInIfOpen().available(); + return n > (Integer.MAX_VALUE - avail) ? Integer.MAX_VALUE : n + avail; + } + + /** + * See the general contract of the {@code mark} method of + * {@code InputStream}. + * + * @param readlimit the maximum limit of bytes that can be read before the + * mark position becomes invalid. + */ + public void mark(int readlimit) { + + implMark(readlimit); + + } + + private void implMark(int readlimit) { + marklimit = readlimit; + markpos = pos; + } + + /** + * See the general contract of the {@code reset} method of + * {@code InputStream}. + *

+ * If {@code markpos} is {@code -1} (no mark has been set or the mark has + * been invalidated), an {@code IOException} is thrown. Otherwise, + * {@code pos} is set equal to {@code markpos}. + * + * @throws IOException if this stream has not been marked or, if the mark + * has been invalidated, or the stream has been closed + * by invoking its {@link #close()} method, or an I/O + * error occurs. + */ + public void reset() throws IOException { + + implReset(); + + } + + private void implReset() throws IOException { + ensureOpen(); + if (markpos < 0) { + throw new IOException("Resetting to invalid mark"); + } + pos = markpos; + } + + /** + * Tests if this input stream supports the {@code mark} and {@code reset} + * methods. The {@code markSupported} method of {@code TempBuffIn} returns + * {@code true}. + * + * @return a {@code boolean} indicating if this stream type supports the + * {@code mark} and {@code reset} methods. + */ + public boolean markSupported() { + return true; + } + + /** + * Closes this input stream and releases any system resources associated + * with the stream. Once the stream has been closed, further read(), + * available(), reset(), or skip() invocations will throw an IOException. + * Closing a previously closed stream has no effect. + * + * @throws IOException if an I/O error occurs. + */ + public void close() throws IOException { + byte[] buffer; + while ((buffer = buf) != null) { + if (U.compareAndSetReference(this, BUF_OFFSET, buffer, null)) { + InputStream input = in; + in = null; + if (input != null) { + input.close(); + } + return; + } + // Else retry in case a new buf was CASed in fill() + } + } + + @Override + public long transferTo(OutputStream out) throws IOException { + Objects.requireNonNull(out, "out"); + + return implTransferTo(out); + + } + + private long implTransferTo(OutputStream out) throws IOException { + if (getClass() == TempBuffIn.class && markpos == -1) { + int avail = count - pos; + if (avail > 0) { + if (isTrusted(out)) { + out.write(getBufIfOpen(), pos, avail); + } else { + // Prevent poisoning and leaking of buf + byte[] buffer = Arrays.copyOfRange(getBufIfOpen(), pos, count); + out.write(buffer); + } + pos = count; + } + try { + return Math.addExact(avail, getInIfOpen().transferTo(out)); + } catch (ArithmeticException ignore) { + return Long.MAX_VALUE; + } + } else { + return super.transferTo(out); + } + } + + /** + * Returns true if this class satisfies the following conditions: + *

    + *
  • does not retain a reference to the {@code byte[]}
  • + *
  • does not leak a reference to the {@code byte[]} to non-trusted + * classes
  • + *
  • does not modify the contents of the {@code byte[]}
  • + *
  • {@code write()} method does not read the contents outside of the + * offset/length bounds
  • + *
+ * + * @return true if this class is trusted + */ + private static boolean isTrusted(OutputStream os) { + var clazz = os.getClass(); + return clazz == ByteArrayOutputStream.class || clazz == FileOutputStream.class + || clazz == PipedOutputStream.class; + } + +} diff --git a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/storage/TempBuffOut.java b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/storage/TempBuffOut.java new file mode 100644 index 000000000..5a20781ef --- /dev/null +++ b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/storage/TempBuffOut.java @@ -0,0 +1,170 @@ +package com.the_qa_company.qendpoint.core.storage; + +import java.io.FilterOutputStream; +import java.io.IOException; +import java.io.OutputStream; +import java.util.Arrays; + +/** + * The class implements a buffered output stream. By setting up such an output + * stream, an application can write bytes to the underlying output stream + * without necessarily causing a call to the underlying system for each byte + * written. + * + * @author Arthur van Hoff + * @since 1.0 + */ +public class TempBuffOut extends FilterOutputStream { + private static final int DEFAULT_INITIAL_BUFFER_SIZE = 512; + private static final int DEFAULT_MAX_BUFFER_SIZE = 8192; + + /** + * The internal buffer where data is stored. + */ + protected byte[] buf; + + /** + * The number of valid bytes in the buffer. This value is always in the + * range {@code 0} through {@code buf.length}; elements {@code buf[0]} + * through {@code buf[count-1]} contain valid byte data. + */ + protected int count; + + /** + * Max size of the internal buffer. + */ + private final int maxBufSize; + + /** + * Returns the buffer size to use when no output buffer size specified. + */ + private static int initialBufferSize() { + return DEFAULT_MAX_BUFFER_SIZE; + } + + /** + * Creates a new buffered output stream. + */ + private TempBuffOut(OutputStream out, int initialSize, int maxSize) { + super(out); + + if (initialSize <= 0) { + throw new IllegalArgumentException("Buffer size <= 0"); + } + + this.buf = new byte[maxSize]; + + this.maxBufSize = maxSize; + } + + /** + * Creates a new buffered output stream to write data to the specified + * underlying output stream. + * + * @param out the underlying output stream. + */ + public TempBuffOut(OutputStream out) { + this(out, initialBufferSize(), DEFAULT_MAX_BUFFER_SIZE); + } + + /** + * Creates a new buffered output stream to write data to the specified + * underlying output stream with the specified buffer size. + * + * @param out the underlying output stream. + * @param size the buffer size. + * @throws IllegalArgumentException if size <= 0. + */ + public TempBuffOut(OutputStream out, int size) { + this(out, size, size); + } + + /** Flush the internal buffer */ + private void flushBuffer() throws IOException { + if (count > 0) { + out.write(buf, 0, count); + count = 0; + } + } + + /** + * Writes the specified byte to this buffered output stream. + * + * @param b the byte to be written. + * @throws IOException if an I/O error occurs. + */ + @Override + public void write(int b) throws IOException { + + implWrite(b); + + } + + private void implWrite(int b) throws IOException { + if (count >= buf.length) { + flushBuffer(); + } + buf[count++] = (byte) b; + } + + /** + * Writes {@code len} bytes from the specified byte array starting at offset + * {@code off} to this buffered output stream. + *

+ * Ordinarily this method stores bytes from the given array into this + * stream's buffer, flushing the buffer to the underlying output stream as + * needed. If the requested length is at least as large as this stream's + * buffer, however, then this method will flush the buffer and write the + * bytes directly to the underlying output stream. Thus redundant + * {@code TempBuffOut}s will not copy data unnecessarily. + * + * @param b the data. + * @param off the start offset in the data. + * @param len the number of bytes to write. + * @throws IOException if an I/O error occurs. + * @throws IndexOutOfBoundsException {@inheritDoc} + */ + @Override + public void write(byte[] b, int off, int len) throws IOException { + + implWrite(b, off, len); + + } + + private void implWrite(byte[] b, int off, int len) throws IOException { + if (len >= maxBufSize) { + /* + * If the request length exceeds the max size of the output buffer, + * flush the output buffer and then write the data directly. In this + * way buffered streams will cascade harmlessly. + */ + flushBuffer(); + out.write(b, off, len); + return; + } + if (len > buf.length - count) { + flushBuffer(); + } + System.arraycopy(b, off, buf, count, len); + count += len; + } + + /** + * Flushes this buffered output stream. This forces any buffered output + * bytes to be written out to the underlying output stream. + * + * @throws IOException if an I/O error occurs. + * @see java.io.FilterOutputStream#out + */ + @Override + public void flush() throws IOException { + + implFlush(); + + } + + private void implFlush() throws IOException { + flushBuffer(); + out.flush(); + } +} diff --git a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/triples/IndexedNode.java b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/triples/IndexedNode.java index 3e08409b0..e237890f7 100644 --- a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/triples/IndexedNode.java +++ b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/triples/IndexedNode.java @@ -50,4 +50,5 @@ public IndexedNode clone() { throw new AssertionError(e); } } + } diff --git a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/triples/impl/BitmapTriplesCat.java b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/triples/impl/BitmapTriplesCat.java index d1f043bf7..4b166783e 100644 --- a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/triples/impl/BitmapTriplesCat.java +++ b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/triples/impl/BitmapTriplesCat.java @@ -16,6 +16,7 @@ import java.io.BufferedOutputStream; import java.io.FileOutputStream; import java.io.IOException; +import java.io.OutputStream; import java.nio.file.Files; import java.nio.file.Paths; @@ -25,6 +26,7 @@ import com.the_qa_company.qendpoint.core.listener.ProgressListener; import com.the_qa_company.qendpoint.core.options.ControlInfo; import com.the_qa_company.qendpoint.core.options.ControlInformation; +import com.the_qa_company.qendpoint.core.storage.TempBuffOut; import com.the_qa_company.qendpoint.core.triples.IteratorTripleID; import com.the_qa_company.qendpoint.core.triples.TripleID; import com.the_qa_company.qendpoint.core.util.BitUtil; @@ -118,7 +120,7 @@ public void cat(IteratorTripleID it, ProgressListener listener) throws IOExcepti vectorY.aggressiveTrimToSize(); vectorZ.trimToSize(); - try (BufferedOutputStream bos = new BufferedOutputStream(new FileOutputStream(location + "triples"))) { + try (OutputStream bos = new TempBuffOut(new FileOutputStream(location + "triples"))) { ControlInfo ci = new ControlInformation(); ci.setType(ControlInfo.Type.TRIPLES); ci.setFormat(HDTVocabulary.TRIPLES_TYPE_BITMAP); diff --git a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/triples/impl/BitmapTriplesIndexFile.java b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/triples/impl/BitmapTriplesIndexFile.java index ab9a6fa33..fe7abc73e 100644 --- a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/triples/impl/BitmapTriplesIndexFile.java +++ b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/triples/impl/BitmapTriplesIndexFile.java @@ -19,6 +19,8 @@ import com.the_qa_company.qendpoint.core.listener.ProgressListener; import com.the_qa_company.qendpoint.core.options.HDTOptions; import com.the_qa_company.qendpoint.core.options.HDTOptionsKeys; +import com.the_qa_company.qendpoint.core.storage.TempBuffIn; +import com.the_qa_company.qendpoint.core.storage.TempBuffOut; import com.the_qa_company.qendpoint.core.triples.TripleID; import com.the_qa_company.qendpoint.core.util.BitUtil; import com.the_qa_company.qendpoint.core.util.concurrent.KWayMerger; @@ -36,6 +38,7 @@ import java.io.Closeable; import java.io.IOException; import java.io.InterruptedIOException; +import java.io.OutputStream; import java.nio.ByteOrder; import java.nio.channels.Channels; import java.nio.channels.FileChannel; @@ -129,7 +132,7 @@ public static BitmapTriplesIndex map(Path file, FileChannel channel, BitmapTripl } } - CountInputStream stream = new CountInputStream(new BufferedInputStream(Channels.newInputStream(channel))); + CountInputStream stream = new CountInputStream(new TempBuffIn(Channels.newInputStream(channel))); stream.skipNBytes(headerSize); String orderCfg = IOUtil.readSizedString(stream, ProgressListener.ignore()); @@ -314,7 +317,7 @@ public static void generateIndex(BitmapTriples triples, Path destination, Triple seqZ.trimToSize(); // saving the index - try (BufferedOutputStream output = new BufferedOutputStream(Files.newOutputStream(destination))) { + try (OutputStream output = new TempBuffOut(Files.newOutputStream(destination))) { output.write(MAGIC); IOUtil.writeLong(output, signature(triples)); diff --git a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/util/LargeFakeDataSetStreamSupplier.java b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/util/LargeFakeDataSetStreamSupplier.java index f2a88a95b..685601cb4 100644 --- a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/util/LargeFakeDataSetStreamSupplier.java +++ b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/util/LargeFakeDataSetStreamSupplier.java @@ -7,6 +7,7 @@ import com.the_qa_company.qendpoint.core.iterator.utils.MapIterator; import com.the_qa_company.qendpoint.core.options.HDTOptions; import com.the_qa_company.qendpoint.core.quad.QuadString; +import com.the_qa_company.qendpoint.core.storage.TempBuffOut; import com.the_qa_company.qendpoint.core.triples.TripleString; import com.the_qa_company.qendpoint.core.util.concurrent.ExceptionThread; import com.the_qa_company.qendpoint.core.util.string.ByteStringUtil; @@ -194,7 +195,7 @@ public void createNTFile(Path file) throws IOException { */ public void createNTFile(Path file, CompressionType compressionType) throws IOException { try (Writer writer = new OutputStreamWriter( - new BufferedOutputStream(compressionType.compress(Files.newOutputStream(file))))) { + new TempBuffOut(compressionType.compress(Files.newOutputStream(file))))) { createNTFile(writer); } } diff --git a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/util/Profiler.java b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/util/Profiler.java index 65e2f8ec8..70403011a 100644 --- a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/util/Profiler.java +++ b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/util/Profiler.java @@ -2,6 +2,8 @@ import com.the_qa_company.qendpoint.core.options.HDTOptions; import com.the_qa_company.qendpoint.core.options.HDTOptionsKeys; +import com.the_qa_company.qendpoint.core.storage.TempBuffIn; +import com.the_qa_company.qendpoint.core.storage.TempBuffOut; import java.io.BufferedInputStream; import java.io.BufferedOutputStream; @@ -48,7 +50,7 @@ public static Profiler getProfilerById(long id) { */ public static Profiler readFromDisk(Path inputPath) throws IOException { Profiler p = new Profiler(""); - try (InputStream is = new BufferedInputStream(Files.newInputStream(inputPath))) { + try (InputStream is = new TempBuffIn(Files.newInputStream(inputPath))) { for (byte b : HEADER) { if (is.read() != b) { throw new IOException("Missing header for the profiling file!"); @@ -286,7 +288,7 @@ public void writeProfiling() throws IOException { * @param outputPath output path */ public void writeToDisk(Path outputPath) throws IOException { - try (OutputStream os = new BufferedOutputStream(Files.newOutputStream(outputPath))) { + try (OutputStream os = new TempBuffOut(Files.newOutputStream(outputPath))) { for (byte b : HEADER) { os.write(b); } diff --git a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/util/RDFInfo.java b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/util/RDFInfo.java index 120bdd19f..9e510da55 100644 --- a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/util/RDFInfo.java +++ b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/util/RDFInfo.java @@ -27,6 +27,7 @@ import com.the_qa_company.qendpoint.core.exceptions.ParserException; import com.the_qa_company.qendpoint.core.options.HDTOptions; import com.the_qa_company.qendpoint.core.rdf.RDFParserCallback; +import com.the_qa_company.qendpoint.core.storage.TempBuffIn; /** * A class for getting basic information about a file @@ -136,7 +137,7 @@ public static float getCompression(HDTOptions specs) { */ public static long countLines(String filename, RDFParserCallback parser, RDFNotation notation) throws IOException, ParserException { - InputStream is = new BufferedInputStream(new FileInputStream(filename)); + InputStream is = new TempBuffIn(new FileInputStream(filename)); try { byte[] c = new byte[1024]; int count = 0; diff --git a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/util/io/CloseSuppressPath.java b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/util/io/CloseSuppressPath.java index 444478166..c3e77ccd3 100644 --- a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/util/io/CloseSuppressPath.java +++ b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/util/io/CloseSuppressPath.java @@ -1,5 +1,8 @@ package com.the_qa_company.qendpoint.core.util.io; +import com.the_qa_company.qendpoint.core.storage.TempBuffIn; +import com.the_qa_company.qendpoint.core.storage.TempBuffOut; + import java.io.BufferedInputStream; import java.io.BufferedOutputStream; import java.io.Closeable; @@ -232,7 +235,7 @@ public Spliterator spliterator() { } public InputStream openInputStream(int bufferSize, OpenOption... options) throws IOException { - return new BufferedInputStream(openInputStream(options), bufferSize); + return new TempBuffIn(openInputStream(options), bufferSize); } public InputStream openInputStream(OpenOption... options) throws IOException { @@ -244,7 +247,7 @@ private OutputStream openOutputStream(OpenOption... options) throws IOException } public OutputStream openOutputStream(int bufferSize, OpenOption... options) throws IOException { - return new BufferedOutputStream(openOutputStream(options), bufferSize); + return new TempBuffOut(openOutputStream(options), bufferSize); } /** diff --git a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/util/io/IOUtil.java b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/util/io/IOUtil.java index 6353515a4..019bf1cae 100644 --- a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/util/io/IOUtil.java +++ b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/util/io/IOUtil.java @@ -25,6 +25,7 @@ import com.the_qa_company.qendpoint.core.listener.ProgressListener; import com.the_qa_company.qendpoint.core.options.HDTOptions; import com.the_qa_company.qendpoint.core.options.HDTOptionsKeys; +import com.the_qa_company.qendpoint.core.storage.TempBuffIn; import com.the_qa_company.qendpoint.core.unsafe.MemoryUtils; import com.the_qa_company.qendpoint.core.unsafe.UnsafeLongArray; import com.the_qa_company.qendpoint.core.util.StringUtil; @@ -394,9 +395,9 @@ public static InputStream getFileInputStream(String fileName, boolean uncompress con.connect(); input = con.getInputStream(); } else if (name.equals("-")) { - input = new BufferedInputStream(System.in); + input = new TempBuffIn(System.in); } else { - input = new BufferedInputStream(new FileInputStream(fileName)); + input = new TempBuffIn(new FileInputStream(fileName)); } if (!skipHandled) { input.skipNBytes(startLen); @@ -615,7 +616,7 @@ public static void copy(InputStream is, OutputStream os, ProgressListener pl, lo } public static void decompressGzip(File src, File trgt) throws IOException { - try (InputStream in = new GZIPInputStream(new BufferedInputStream(new FileInputStream(src)))) { + try (InputStream in = new GZIPInputStream(new TempBuffIn(new FileInputStream(src)))) { Files.copy(in, trgt.toPath()); } } diff --git a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/util/listener/MultiThreadListenerConsole.java b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/util/listener/MultiThreadListenerConsole.java index fbff6b8c7..0e5ba1023 100644 --- a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/util/listener/MultiThreadListenerConsole.java +++ b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/util/listener/MultiThreadListenerConsole.java @@ -194,11 +194,12 @@ synchronized private void render(String ln) { message.append("\r"); // go back each line of the thread message - if (previous != 0) { - for (int i = 0; i < previous; i++) { - message.append(goBackNLine(1)).append(ERASE_LINE); - } - } +// if (previous != 0) { +// for (int i = 0; i < previous; i++) { +// message.append(goBackNLine(1)).append(ERASE_LINE); +// } +// } + message.append("\033[H\033[2J"); if (ln != null) { message.append(ln).append("\n"); diff --git a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/util/nsd/NamespaceData.java b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/util/nsd/NamespaceData.java index c4afaaf8b..d8d4674dc 100644 --- a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/util/nsd/NamespaceData.java +++ b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/util/nsd/NamespaceData.java @@ -3,6 +3,8 @@ import com.the_qa_company.qendpoint.core.compact.integer.VByte; import com.the_qa_company.qendpoint.core.listener.ProgressListener; import com.the_qa_company.qendpoint.core.storage.QEPCoreException; +import com.the_qa_company.qendpoint.core.storage.TempBuffIn; +import com.the_qa_company.qendpoint.core.storage.TempBuffOut; import com.the_qa_company.qendpoint.core.util.crc.CRC32; import com.the_qa_company.qendpoint.core.util.crc.CRCOutputStream; import com.the_qa_company.qendpoint.core.util.io.IOUtil; @@ -140,7 +142,7 @@ public void sync() throws QEPCoreException { */ public void load() throws QEPCoreException { synchronized (syncObject) { - try (InputStream is = new BufferedInputStream(Files.newInputStream(location))) { + try (InputStream is = new TempBuffIn(Files.newInputStream(location))) { // check the binary magic byte[] header = is.readNBytes(MAGIC.length + 1); if (header.length < MAGIC.length + 1) { @@ -193,7 +195,7 @@ public void save(boolean onlyIfUpdated) throws QEPCoreException { return; // not updated } ProgressListener pl = ProgressListener.ignore(); - try (OutputStream osh = new BufferedOutputStream(Files.newOutputStream(location))) { + try (OutputStream osh = new TempBuffOut(Files.newOutputStream(location))) { // write magic and version osh.write(MAGIC); osh.write(NS_VERSION); diff --git a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/util/string/ByteString.java b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/util/string/ByteString.java index 50912763d..9f0bca93e 100644 --- a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/util/string/ByteString.java +++ b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/util/string/ByteString.java @@ -1,5 +1,10 @@ package com.the_qa_company.qendpoint.core.util.string; +import jdk.incubator.vector.ShortVector; +import jdk.incubator.vector.VectorMask; +import jdk.incubator.vector.VectorOperators; +import jdk.incubator.vector.VectorSpecies; + import java.util.Arrays; /** @@ -44,19 +49,42 @@ static ByteString copy(CharSequence csq) { */ byte[] getBuffer(); - /* - * (non-Javadoc) - * @see java.lang.Comparable#compareTo(java.lang.Object) - */ @Override default int compareTo(ByteString other) { - - int compare = Arrays.compare(getBuffer(), other.getBuffer()); - if (compare != 0) { - return compare; + int n = Math.min(length(), other.length()); + int k = 0; + while (k < n) { + char c1 = charAt(k); + char c2 = other.charAt(k); + if (c1 != c2) { + return c1 - c2; + } + k++; } + return length() - other.length(); + } + +// /* +// * (non-Javadoc) +// * @see java.lang.Comparable#compareTo(java.lang.Object) +// */ +// @Override +// default int compareTo(ByteString other) { // int n = Math.min(length(), other.length()); -// int k = 0; +// +// if (n == 0) { +// return length() - other.length(); +// } +// +// byte[] buffer = getBuffer(); +// byte[] buffer1 = other.getBuffer(); +// +// int mismatch = Arrays.mismatch(buffer, buffer1); +// if (mismatch == -1) { +// return length() - other.length(); +// } +// +// int k = mismatch; // while (k < n) { // char c1 = charAt(k); // char c2 = other.charAt(k); @@ -65,8 +93,8 @@ default int compareTo(ByteString other) { // } // k++; // } - return length() - other.length(); - } +// return length() - other.length(); +// } @Override ByteString subSequence(int start, int end); diff --git a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/util/string/ByteStringUtil.java b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/util/string/ByteStringUtil.java index 55b9ab0c9..d6231739a 100644 --- a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/util/string/ByteStringUtil.java +++ b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/util/string/ByteStringUtil.java @@ -121,20 +121,56 @@ public static int longestCommonPrefix(CharSequence str1, CharSequence str2) { } public static int longestCommonPrefix(CharSequence str1, CharSequence str2, int from) { + int len = Math.min(str1.length(), str2.length()); + int delta = from; + while (delta < len && str1.charAt(delta) == str2.charAt(delta)) { + delta++; + } + return delta - from; + } + + public static int longestCommonPrefixNew(CharSequence str1, CharSequence str2, int from) { + + int len = Math.min(str1.length(), str2.length()); + if (from >= len) { + return 0; + } +// +// if (str1.charAt(from) != str2.charAt(from)) { +// return 0; +// } if (str1 instanceof ByteString && str2 instanceof ByteString) { + byte[] buffer = ((ByteString) str1).getBuffer(); byte[] buffer2 = ((ByteString) str2).getBuffer(); // System.out.println("mismatch: " + i); - int missmatch = Arrays.mismatch(buffer, from, buffer.length, buffer2, from, buffer2.length); + int missmatch = Arrays.mismatch(buffer, from, len, buffer2, from, len); if (missmatch == -1) { - return Math.min(buffer.length, buffer2.length) - from; + return len - from; + } else { + return missmatch; } - return missmatch - from; +// int delta = from; +// while (delta < len && str1.charAt(delta) == str2.charAt(delta)) { +// delta++; +// } +// // System.out.println("i: " + i); +// int i = delta - from; +// +// int i1 = missmatch-from; +// +// if (i != i1) { +//// longestCommonPrefix(str1, str2, from); +// throw new AssertionError("Mismatch: " + i + " " + i1); +// } +// +// return i1; + +// return missmatch - from; } - int len = Math.min(str1.length(), str2.length()); int delta = from; while (delta < len && str1.charAt(delta) == str2.charAt(delta)) { delta++; diff --git a/qendpoint-core/src/test/java/com/the_qa_company/qendpoint/core/compact/bitmap/MultiRoaringBitmapTest.java b/qendpoint-core/src/test/java/com/the_qa_company/qendpoint/core/compact/bitmap/MultiRoaringBitmapTest.java index 221732cf0..b736dda25 100644 --- a/qendpoint-core/src/test/java/com/the_qa_company/qendpoint/core/compact/bitmap/MultiRoaringBitmapTest.java +++ b/qendpoint-core/src/test/java/com/the_qa_company/qendpoint/core/compact/bitmap/MultiRoaringBitmapTest.java @@ -1,6 +1,8 @@ package com.the_qa_company.qendpoint.core.compact.bitmap; import com.the_qa_company.qendpoint.core.listener.ProgressListener; +import com.the_qa_company.qendpoint.core.storage.TempBuffIn; +import com.the_qa_company.qendpoint.core.storage.TempBuffOut; import com.the_qa_company.qendpoint.core.util.io.AbstractMapMemoryTest; import org.apache.commons.io.file.PathUtils; import org.junit.After; @@ -12,6 +14,8 @@ import java.io.BufferedInputStream; import java.io.BufferedOutputStream; import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; import java.nio.file.Files; import java.nio.file.Path; import java.util.Random; @@ -52,7 +56,7 @@ public void serialSyncTest() throws IOException { map.set(0, 90, true); } - try (BufferedInputStream stream = new BufferedInputStream(Files.newInputStream(output)); + try (InputStream stream = new TempBuffIn(Files.newInputStream(output)); MultiRoaringBitmap map = MultiRoaringBitmap.load(stream)) { for (int i = 0; i < 100; i++) { switch (i) { @@ -102,7 +106,7 @@ public void largeSerialSyncTest() throws IOException { assertTrue(map.access(layer, position)); } - try (BufferedOutputStream out = new BufferedOutputStream(Files.newOutputStream(output))) { + try (OutputStream out = new TempBuffOut(Files.newOutputStream(output))) { map.save(out, ProgressListener.ignore()); } } diff --git a/qendpoint-core/src/test/java/com/the_qa_company/qendpoint/core/dictionary/impl/kcat/KCatMergerTest.java b/qendpoint-core/src/test/java/com/the_qa_company/qendpoint/core/dictionary/impl/kcat/KCatMergerTest.java index b5bd98f34..77996f0b6 100644 --- a/qendpoint-core/src/test/java/com/the_qa_company/qendpoint/core/dictionary/impl/kcat/KCatMergerTest.java +++ b/qendpoint-core/src/test/java/com/the_qa_company/qendpoint/core/dictionary/impl/kcat/KCatMergerTest.java @@ -20,6 +20,8 @@ import com.the_qa_company.qendpoint.core.listener.ProgressListener; import com.the_qa_company.qendpoint.core.options.HDTOptions; import com.the_qa_company.qendpoint.core.options.HDTOptionsKeys; +import com.the_qa_company.qendpoint.core.storage.TempBuffIn; +import com.the_qa_company.qendpoint.core.storage.TempBuffOut; import com.the_qa_company.qendpoint.core.triples.IteratorTripleString; import com.the_qa_company.qendpoint.core.triples.TripleString; import com.the_qa_company.qendpoint.core.util.LargeFakeDataSetStreamSupplier; @@ -147,7 +149,7 @@ public void mergerTest() throws ParserException, IOException, InterruptedExcepti merger.startMerger(); // create DictionaryPrivate dict = merger.buildDictionary(); - try (OutputStream stream = new BufferedOutputStream(Files.newOutputStream(dictFile))) { + try (OutputStream stream = new TempBuffOut(Files.newOutputStream(dictFile))) { writeSection(dict.getShared(), stream); writeSection(dict.getSubjects(), stream); writeSection(dict.getPredicates(), stream); @@ -238,7 +240,7 @@ public void mergerTest() throws ParserException, IOException, InterruptedExcepti } } } - try (InputStream stream = new BufferedInputStream(Files.newInputStream(dictFile))) { + try (InputStream stream = new TempBuffIn(Files.newInputStream(dictFile))) { // read the sections try (DictionarySection sh = loadSection(stream); DictionarySection su = loadSection(stream); @@ -481,7 +483,7 @@ public void diffMergerTest() throws ParserException, IOException, InterruptedExc merger.startMerger(); // create DictionaryPrivate dict = merger.buildDictionary(); - try (OutputStream stream = new BufferedOutputStream(Files.newOutputStream(dictFile))) { + try (OutputStream stream = new TempBuffOut(Files.newOutputStream(dictFile))) { writeSection(dict.getShared(), stream); writeSection(dict.getSubjects(), stream); writeSection(dict.getPredicates(), stream); @@ -499,7 +501,7 @@ public void diffMergerTest() throws ParserException, IOException, InterruptedExc } } } - try (InputStream stream = new BufferedInputStream(Files.newInputStream(dictFile))) { + try (InputStream stream = new TempBuffIn(Files.newInputStream(dictFile))) { // read the sections try (DictionarySection sh = loadSection(stream); DictionarySection su = loadSection(stream); diff --git a/qendpoint-core/src/test/java/com/the_qa_company/qendpoint/core/dictionary/impl/section/WriteDictionarySectionTest.java b/qendpoint-core/src/test/java/com/the_qa_company/qendpoint/core/dictionary/impl/section/WriteDictionarySectionTest.java index 22d1168f7..187179aaf 100644 --- a/qendpoint-core/src/test/java/com/the_qa_company/qendpoint/core/dictionary/impl/section/WriteDictionarySectionTest.java +++ b/qendpoint-core/src/test/java/com/the_qa_company/qendpoint/core/dictionary/impl/section/WriteDictionarySectionTest.java @@ -3,6 +3,7 @@ import com.the_qa_company.qendpoint.core.iterator.utils.MapIterator; import com.the_qa_company.qendpoint.core.listener.ProgressListener; import com.the_qa_company.qendpoint.core.options.HDTOptions; +import com.the_qa_company.qendpoint.core.storage.TempBuffOut; import com.the_qa_company.qendpoint.core.util.LargeFakeDataSetStreamSupplier; import com.the_qa_company.qendpoint.core.util.string.ByteString; import org.apache.commons.io.file.PathUtils; @@ -49,10 +50,10 @@ public void appenderTest() throws IOException { } Path t1Save = dir.resolve("t1.save"); Path t2Save = dir.resolve("t2.save"); - try (OutputStream os = new BufferedOutputStream(Files.newOutputStream(t1Save))) { + try (OutputStream os = new TempBuffOut(Files.newOutputStream(t1Save))) { section1.save(os, ProgressListener.ignore()); } - try (OutputStream os = new BufferedOutputStream(Files.newOutputStream(t2Save))) { + try (OutputStream os = new TempBuffOut(Files.newOutputStream(t2Save))) { section2.save(os, ProgressListener.ignore()); } diff --git a/qendpoint-core/src/test/java/com/the_qa_company/qendpoint/core/rdf/parsers/TarTest.java b/qendpoint-core/src/test/java/com/the_qa_company/qendpoint/core/rdf/parsers/TarTest.java index 6f6cbf1b4..a54b3a876 100644 --- a/qendpoint-core/src/test/java/com/the_qa_company/qendpoint/core/rdf/parsers/TarTest.java +++ b/qendpoint-core/src/test/java/com/the_qa_company/qendpoint/core/rdf/parsers/TarTest.java @@ -5,6 +5,7 @@ import java.io.InputStream; import java.util.zip.GZIPInputStream; +import com.the_qa_company.qendpoint.core.storage.TempBuffIn; import org.apache.commons.compress.archivers.tar.TarArchiveEntry; import org.apache.commons.compress.archivers.tar.TarArchiveInputStream; import com.the_qa_company.qendpoint.core.util.io.CountInputStream; @@ -14,9 +15,9 @@ public class TarTest { public static void main(String[] args) throws Throwable { - InputStream input = new CountInputStream(new BufferedInputStream( - new GZIPInputStream(new FileInputStream("/Users/mck/rdf/dataset/tgztest.tar.gz")))); -// InputStream input = new CountInputStream(new BufferedInputStream(new FileInputStream("/Users/mck/rdf/dataset/tgztest.tar"))); + InputStream input = new CountInputStream( + new TempBuffIn(new GZIPInputStream(new FileInputStream("/Users/mck/rdf/dataset/tgztest.tar.gz")))); +// InputStream input = new CountInputStream(new TempBuffIn(new FileInputStream("/Users/mck/rdf/dataset/tgztest.tar"))); final TarArchiveInputStream debInputStream = new TarArchiveInputStream(input); TarArchiveEntry entry; diff --git a/qendpoint-core/src/test/java/com/the_qa_company/qendpoint/core/triples/impl/BitmapQuadTriplesTest.java b/qendpoint-core/src/test/java/com/the_qa_company/qendpoint/core/triples/impl/BitmapQuadTriplesTest.java index aeeb40c51..1cd4a7714 100644 --- a/qendpoint-core/src/test/java/com/the_qa_company/qendpoint/core/triples/impl/BitmapQuadTriplesTest.java +++ b/qendpoint-core/src/test/java/com/the_qa_company/qendpoint/core/triples/impl/BitmapQuadTriplesTest.java @@ -4,6 +4,8 @@ import com.the_qa_company.qendpoint.core.enums.TripleComponentOrder; import com.the_qa_company.qendpoint.core.listener.ProgressListener; import com.the_qa_company.qendpoint.core.options.ControlInformation; +import com.the_qa_company.qendpoint.core.storage.TempBuffIn; +import com.the_qa_company.qendpoint.core.storage.TempBuffOut; import com.the_qa_company.qendpoint.core.triples.IteratorTripleID; import com.the_qa_company.qendpoint.core.triples.TripleID; import com.the_qa_company.qendpoint.core.util.io.AbstractMapMemoryTest; @@ -145,14 +147,14 @@ public void triplesTest() throws IOException { assertEquals(it2.next(), it.next()); } - try (OutputStream stream = new BufferedOutputStream(Files.newOutputStream(path))) { + try (OutputStream stream = new TempBuffOut(Files.newOutputStream(path))) { triples.save(stream, new ControlInformation(), ProgressListener.ignore()); } } // load try (BitmapQuadTriples triples = new BitmapQuadTriples()) { - try (InputStream stream = new BufferedInputStream(Files.newInputStream(path))) { + try (InputStream stream = new TempBuffIn(Files.newInputStream(path))) { ControlInformation ci = new ControlInformation(); ci.load(stream); triples.load(stream, ci, ProgressListener.ignore()); @@ -168,7 +170,7 @@ public void triplesTest() throws IOException { // map try (BitmapQuadTriples triples = new BitmapQuadTriples()) { - try (InputStream stream = new BufferedInputStream(Files.newInputStream(path))) { + try (InputStream stream = new TempBuffIn(Files.newInputStream(path))) { CountInputStream cstream = new CountInputStream(stream); triples.mapFromFile(cstream, path.toFile(), ProgressListener.ignore()); } diff --git a/qendpoint-core/src/test/java/com/the_qa_company/qendpoint/core/util/LargeFakeDataSetStreamSupplierTest.java b/qendpoint-core/src/test/java/com/the_qa_company/qendpoint/core/util/LargeFakeDataSetStreamSupplierTest.java index c69a99efe..30b09b390 100644 --- a/qendpoint-core/src/test/java/com/the_qa_company/qendpoint/core/util/LargeFakeDataSetStreamSupplierTest.java +++ b/qendpoint-core/src/test/java/com/the_qa_company/qendpoint/core/util/LargeFakeDataSetStreamSupplierTest.java @@ -10,6 +10,7 @@ import com.the_qa_company.qendpoint.core.options.HDTSpecification; import com.the_qa_company.qendpoint.core.rdf.RDFParserCallback; import com.the_qa_company.qendpoint.core.rdf.RDFParserFactory; +import com.the_qa_company.qendpoint.core.storage.TempBuffIn; import com.the_qa_company.qendpoint.core.triples.TripleString; import com.the_qa_company.qendpoint.core.triples.impl.utils.HDTTestUtils; import org.junit.Rule; @@ -184,8 +185,8 @@ public void mergeTest() throws IOException, ParserException, NotFoundException { RDFParserCallback parser = RDFParserFactory.getParserCallback(RDFNotation.NTRIPLES, HDTOptions.of(Map.of(HDTOptionsKeys.NT_SIMPLE_PARSER_KEY, "true"))); try { - try (InputStream stream = new BufferedInputStream(Files.newInputStream(p12)); - InputStream stream2 = new BufferedInputStream(Files.newInputStream(p3)); + try (InputStream stream = new TempBuffIn(Files.newInputStream(p12)); + InputStream stream2 = new TempBuffIn(Files.newInputStream(p3)); PipedCopyIterator it1 = RDFParserFactory.readAsIterator(parser, stream, "http://w", true, RDFNotation.NTRIPLES); PipedCopyIterator it2 = RDFParserFactory.readAsIterator(parser, stream2, "http://w", diff --git a/qendpoint-store/pom.xml b/qendpoint-store/pom.xml index 62bcdf644..9053c9843 100644 --- a/qendpoint-store/pom.xml +++ b/qendpoint-store/pom.xml @@ -183,6 +183,14 @@ ${java.source.version} ${java.target.version} + + --add-exports + java.base/jdk.internal.misc=ALL-UNNAMED + --add-exports + java.base/jdk.internal.util=ALL-UNNAMED + --add-modules + jdk.incubator.vector + diff --git a/qendpoint-store/src/main/java/com/the_qa_company/qendpoint/compiler/SailCompiler.java b/qendpoint-store/src/main/java/com/the_qa_company/qendpoint/compiler/SailCompiler.java index d9778a8bd..85289a629 100644 --- a/qendpoint-store/src/main/java/com/the_qa_company/qendpoint/compiler/SailCompiler.java +++ b/qendpoint-store/src/main/java/com/the_qa_company/qendpoint/compiler/SailCompiler.java @@ -8,6 +8,7 @@ import com.the_qa_company.qendpoint.compiler.source.EmptyTripleSourceGetter; import com.the_qa_company.qendpoint.compiler.source.ModelTripleSourceGetter; import com.the_qa_company.qendpoint.compiler.source.SailTripleSourceModel; +import com.the_qa_company.qendpoint.core.storage.TempBuffIn; import com.the_qa_company.qendpoint.utils.sail.linked.LinkedSail; import org.eclipse.rdf4j.common.iteration.CloseableIteration; import org.eclipse.rdf4j.model.IRI; @@ -133,7 +134,7 @@ public void load(Path rdfFile) throws IOException { RDFFormat format = Rio.getParserFormatForFileName(rdfFile.getFileName().toString()) .orElseThrow(() -> new IllegalArgumentException("Can't find parser for file: " + rdfFile)); - try (InputStream stream = new BufferedInputStream(Files.newInputStream(rdfFile))) { + try (InputStream stream = new TempBuffIn(Files.newInputStream(rdfFile))) { load(stream, format); } } diff --git a/qendpoint-store/src/main/java/com/the_qa_company/qendpoint/store/EndpointStore.java b/qendpoint-store/src/main/java/com/the_qa_company/qendpoint/store/EndpointStore.java index 62a7a488d..c06882ca9 100644 --- a/qendpoint-store/src/main/java/com/the_qa_company/qendpoint/store/EndpointStore.java +++ b/qendpoint-store/src/main/java/com/the_qa_company/qendpoint/store/EndpointStore.java @@ -11,6 +11,7 @@ import com.the_qa_company.qendpoint.core.hdt.HDTManager; import com.the_qa_company.qendpoint.core.options.HDTOptions; import com.the_qa_company.qendpoint.core.options.HDTOptionsKeys; +import com.the_qa_company.qendpoint.core.storage.TempBuffOut; import com.the_qa_company.qendpoint.core.triples.IteratorTripleID; import com.the_qa_company.qendpoint.core.triples.TripleID; import com.the_qa_company.qendpoint.core.triples.TripleString; @@ -700,7 +701,7 @@ public void initTempDump(boolean isRestarting) { if (!file.exists()) { Files.createFile(file.toPath()); } - OutputStream rdfWriterTempTriplesOut = new BufferedOutputStream(new FileOutputStream(file, isRestarting)); + OutputStream rdfWriterTempTriplesOut = new TempBuffOut(new FileOutputStream(file, isRestarting)); this.rdfWriterTempTriples = graph ? new NQuadsWriter(rdfWriterTempTriplesOut) : new NTriplesWriter(rdfWriterTempTriplesOut); this.rdfWriterTempTriples.startRDF(); diff --git a/qendpoint-store/src/main/java/com/the_qa_company/qendpoint/store/MergeRunnable.java b/qendpoint-store/src/main/java/com/the_qa_company/qendpoint/store/MergeRunnable.java index 005b75ea6..639886317 100644 --- a/qendpoint-store/src/main/java/com/the_qa_company/qendpoint/store/MergeRunnable.java +++ b/qendpoint-store/src/main/java/com/the_qa_company/qendpoint/store/MergeRunnable.java @@ -4,6 +4,7 @@ import com.the_qa_company.qendpoint.core.compact.bitmap.MultiLayerBitmapWrapper; import com.the_qa_company.qendpoint.core.dictionary.Dictionary; import com.the_qa_company.qendpoint.core.enums.TripleComponentOrder; +import com.the_qa_company.qendpoint.core.storage.TempBuffOut; import com.the_qa_company.qendpoint.store.exception.EndpointStoreException; import com.the_qa_company.qendpoint.utils.BitArrayDisk; import com.the_qa_company.qendpoint.utils.OverrideHDTOptions; @@ -925,7 +926,7 @@ private void createHDTDump(String rdfInput, String hdtOutput) throws IOException } private void writeTempFile(RepositoryConnection connection, String file, boolean graph) throws IOException { - try (OutputStream out = new BufferedOutputStream(new FileOutputStream(file))) { + try (OutputStream out = new TempBuffOut(new FileOutputStream(file))) { RDFWriter writer = Rio.createWriter( endpoint.getHdt().getDictionary().supportGraphs() ? RDFFormat.NQUADS : RDFFormat.NTRIPLES, out); RepositoryResult repositoryResult = connection.getStatements(null, null, null, false); diff --git a/qendpoint-store/src/main/java/com/the_qa_company/qendpoint/tools/QEPSearch.java b/qendpoint-store/src/main/java/com/the_qa_company/qendpoint/tools/QEPSearch.java index e96b6b764..fe06f33b2 100644 --- a/qendpoint-store/src/main/java/com/the_qa_company/qendpoint/tools/QEPSearch.java +++ b/qendpoint-store/src/main/java/com/the_qa_company/qendpoint/tools/QEPSearch.java @@ -17,6 +17,7 @@ import com.the_qa_company.qendpoint.core.options.HDTOptions; import com.the_qa_company.qendpoint.core.options.HDTOptionsKeys; import com.the_qa_company.qendpoint.core.rdf.parsers.RDFDeltaFileParser; +import com.the_qa_company.qendpoint.core.storage.TempBuffIn; import com.the_qa_company.qendpoint.core.tools.HDTVerify; import com.the_qa_company.qendpoint.core.triples.IteratorTripleString; import com.the_qa_company.qendpoint.core.triples.TripleString; @@ -1028,7 +1029,7 @@ private void executeDeltaFile() throws IOException { HDTOptionsKeys.PARSER_DELTAFILE_NO_EXCEPTION, true); try (RDFDeltaFileParser.DeltaFileReader reader = new RDFDeltaFileParser.DeltaFileReader( - new BufferedInputStream(Files.newInputStream(file)), spec)) { + new TempBuffIn(Files.newInputStream(file)), spec)) { console.printLine(console.color(5, 5, 1) + "files .. " + console.colorReset() + reader.getSize()); console.printLine(console.color(5, 5, 1) + "start .. " + console.colorReset() + reader.getStart()); diff --git a/qendpoint-store/src/main/java/com/the_qa_company/qendpoint/utils/BitArrayDisk.java b/qendpoint-store/src/main/java/com/the_qa_company/qendpoint/utils/BitArrayDisk.java index efc354272..c29271cd9 100644 --- a/qendpoint-store/src/main/java/com/the_qa_company/qendpoint/utils/BitArrayDisk.java +++ b/qendpoint-store/src/main/java/com/the_qa_company/qendpoint/utils/BitArrayDisk.java @@ -1,5 +1,6 @@ package com.the_qa_company.qendpoint.utils; +import com.the_qa_company.qendpoint.core.storage.TempBuffIn; import com.the_qa_company.qendpoint.store.exception.EndpointStoreException; import org.eclipse.rdf4j.common.io.NioFile; import com.the_qa_company.qendpoint.core.compact.bitmap.ModifiableBitmap; @@ -140,8 +141,7 @@ private void initWordsArray(long nbits) throws IOException { int lastNonZero = -1; // read previous values - try (BufferedInputStream is = new BufferedInputStream( - Files.newInputStream(this.output.getFile().toPath()))) { + try (InputStream is = new TempBuffIn(Files.newInputStream(this.output.getFile().toPath()))) { // skip header is.skipNBytes(8); for (int i = 0; i < this.words.length; i++) { diff --git a/qendpoint-store/src/main/java/com/the_qa_company/qendpoint/utils/FormatUtils.java b/qendpoint-store/src/main/java/com/the_qa_company/qendpoint/utils/FormatUtils.java index 74aa956bd..038064cb6 100644 --- a/qendpoint-store/src/main/java/com/the_qa_company/qendpoint/utils/FormatUtils.java +++ b/qendpoint-store/src/main/java/com/the_qa_company/qendpoint/utils/FormatUtils.java @@ -2,6 +2,7 @@ import com.the_qa_company.qendpoint.core.options.ControlInfo; import com.the_qa_company.qendpoint.core.options.ControlInformation; +import com.the_qa_company.qendpoint.core.storage.TempBuffIn; import org.eclipse.rdf4j.query.resultio.QueryResultFormat; import org.eclipse.rdf4j.query.resultio.TupleQueryResultWriterRegistry; import org.eclipse.rdf4j.rio.RDFFormat; @@ -87,7 +88,7 @@ public static Optional getRDFWriterFormat(String acceptHeader) { */ public static byte[] readCookie(Path file, int size) { try (InputStream stream = Files.newInputStream(file)) { - return (size > 0x1000 ? new BufferedInputStream(stream) : stream).readNBytes(size); + return (size > 0x1000 ? new TempBuffIn(stream) : stream).readNBytes(size); } catch (IOException e) { return new byte[0]; } diff --git a/qendpoint-store/src/test/java/com/the_qa_company/qendpoint/store/MergeRestartTest.java b/qendpoint-store/src/test/java/com/the_qa_company/qendpoint/store/MergeRestartTest.java index d7c4ba3e0..69c86dfd4 100644 --- a/qendpoint-store/src/test/java/com/the_qa_company/qendpoint/store/MergeRestartTest.java +++ b/qendpoint-store/src/test/java/com/the_qa_company/qendpoint/store/MergeRestartTest.java @@ -2,6 +2,7 @@ import com.the_qa_company.qendpoint.core.compact.bitmap.MultiLayerBitmapWrapper; import com.the_qa_company.qendpoint.core.enums.TripleComponentOrder; +import com.the_qa_company.qendpoint.core.storage.TempBuffOut; import com.the_qa_company.qendpoint.utils.BitArrayDisk; import org.apache.commons.io.FileUtils; import org.eclipse.rdf4j.common.iteration.CloseableIteration; @@ -888,8 +889,7 @@ private void executeTestRemoveHDT(File out, SailRepository repo, int id, int cou connection.remove(stm); }); writeInfoCount(out, count); - try (OutputStream buff = new BufferedOutputStream( - new FileOutputStream(out.getAbsolutePath() + ".delta", true))) { + try (OutputStream buff = new TempBuffOut(new FileOutputStream(out.getAbsolutePath() + ".delta", true))) { buff.write(("REM HDT " + id + " / " + count + "\n").getBytes(StandardCharsets.UTF_8)); } @@ -913,8 +913,7 @@ private void executeTestRemoveRDF(File out, SailRepository repo, int id, int cou connection.remove(stm); }); writeInfoCount(out, count); - try (OutputStream buff = new BufferedOutputStream( - new FileOutputStream(out.getAbsolutePath() + ".delta", true))) { + try (OutputStream buff = new TempBuffOut(new FileOutputStream(out.getAbsolutePath() + ".delta", true))) { buff.write(("REM RDF " + id + " / " + count + "\n").getBytes(StandardCharsets.UTF_8)); } } @@ -937,8 +936,7 @@ private void executeTestAddRDF(File out, SailRepository repo, int id, int count) connection.add(stm); }); writeInfoCount(out, count); - try (OutputStream buff = new BufferedOutputStream( - new FileOutputStream(out.getAbsolutePath() + ".delta", true))) { + try (OutputStream buff = new TempBuffOut(new FileOutputStream(out.getAbsolutePath() + ".delta", true))) { buff.write(("ADD RDF " + id + " / " + count + "\n").getBytes(StandardCharsets.UTF_8)); } } @@ -962,8 +960,7 @@ private void executeTestAddHDT(File out, SailRepository repo, int id, int count) connection.add(stm); }); writeInfoCount(out, count); - try (OutputStream buff = new BufferedOutputStream( - new FileOutputStream(out.getAbsolutePath() + ".delta", true))) { + try (OutputStream buff = new TempBuffOut(new FileOutputStream(out.getAbsolutePath() + ".delta", true))) { buff.write(("ADD HDT " + id + " / " + count + "\n").getBytes(StandardCharsets.UTF_8)); } } diff --git a/qendpoint-store/src/test/java/com/the_qa_company/qendpoint/store/Utility.java b/qendpoint-store/src/test/java/com/the_qa_company/qendpoint/store/Utility.java index 3d551ad79..632138faa 100644 --- a/qendpoint-store/src/test/java/com/the_qa_company/qendpoint/store/Utility.java +++ b/qendpoint-store/src/test/java/com/the_qa_company/qendpoint/store/Utility.java @@ -1,5 +1,6 @@ package com.the_qa_company.qendpoint.store; +import com.the_qa_company.qendpoint.core.storage.TempBuffOut; import org.eclipse.rdf4j.model.IRI; import org.eclipse.rdf4j.model.Resource; import org.eclipse.rdf4j.model.Statement; @@ -160,7 +161,7 @@ public static Statement getFakeStatement(ValueFactory vf, int id) { private static void writeBigIndex(File file) throws IOException { ValueFactory vf = new MemValueFactory(); - try (OutputStream out = new BufferedOutputStream(new FileOutputStream(file))) { + try (OutputStream out = new TempBuffOut(new FileOutputStream(file))) { RDFWriter writer = Rio.createWriter(RDFFormat.NTRIPLES, out); writer.startRDF(); for (int i = 1; i <= COUNT; i++) { From f8e681399abfd37df68152546a88524930a6de9c Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ha=CC=8Avard=20Ottestad?= Date: Wed, 12 Feb 2025 13:02:16 +0100 Subject: [PATCH 10/28] wip --- .../iterator/utils/PipedCopyIterator.java | 99 +----- .../utils/PipedCopyIteratorUnordered.java | 324 ++++++++++++++++++ .../qendpoint/core/rdf/RDFParserFactory.java | 11 +- 3 files changed, 336 insertions(+), 98 deletions(-) create mode 100644 qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/iterator/utils/PipedCopyIteratorUnordered.java diff --git a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/iterator/utils/PipedCopyIterator.java b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/iterator/utils/PipedCopyIterator.java index 1343e0536..ab9aa05a0 100644 --- a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/iterator/utils/PipedCopyIterator.java +++ b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/iterator/utils/PipedCopyIterator.java @@ -5,8 +5,6 @@ import java.util.Iterator; import java.util.Objects; import java.util.concurrent.ArrayBlockingQueue; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicInteger; import java.util.function.Function; /** @@ -112,19 +110,7 @@ public T get() { } } - private final ArrayBlockingQueue>[] queue = new ArrayBlockingQueue[] { - new ArrayBlockingQueue<>(16 * 1024), new ArrayBlockingQueue<>(16 * 1024), - new ArrayBlockingQueue<>(16 * 1024), new ArrayBlockingQueue<>(16 * 1024), - new ArrayBlockingQueue<>(16 * 1024), new ArrayBlockingQueue<>(16 * 1024), - new ArrayBlockingQueue<>(16 * 1024), new ArrayBlockingQueue<>(16 * 1024), - new ArrayBlockingQueue<>(16 * 1024), new ArrayBlockingQueue<>(16 * 1024), - new ArrayBlockingQueue<>(16 * 1024), new ArrayBlockingQueue<>(16 * 1024), - new ArrayBlockingQueue<>(16 * 1024), new ArrayBlockingQueue<>(16 * 1024), - new ArrayBlockingQueue<>(16 * 1024), new ArrayBlockingQueue<>(16 * 1024), - new ArrayBlockingQueue<>(16 * 1024), new ArrayBlockingQueue<>(16 * 1024), - new ArrayBlockingQueue<>(16 * 1024), new ArrayBlockingQueue<>(16 * 1024), - new ArrayBlockingQueue<>(16 * 1024), new ArrayBlockingQueue<>(16 * 1024), - new ArrayBlockingQueue<>(16 * 1024), new ArrayBlockingQueue<>(16 * 1024), }; + private final ArrayBlockingQueue> queue = new ArrayBlockingQueue<>(16); private T next; private boolean end; @@ -132,10 +118,6 @@ public T get() { private Thread thread; - AtomicInteger indexHasNext = new AtomicInteger(0); - - volatile ArrayBlockingQueue> focusQueue; - @Override public boolean hasNext() { if (end) { @@ -147,13 +129,7 @@ public boolean hasNext() { QueueObject obj; try { - obj = useFocusQueue(); - - if (obj == null) { - - obj = useThreadBasedQueue(); - } - + obj = queue.take(); } catch (InterruptedException e) { throw new PipedIteratorException("Can't read pipe", e); } @@ -169,51 +145,6 @@ public boolean hasNext() { return true; } - private QueueObject useThreadBasedQueue() throws InterruptedException { - QueueObject obj; - int i = Thread.currentThread().hashCode(); - obj = queue[i % queue.length].poll(); - if (obj == null) { - obj = iterateThroughAllQueues(obj); - } else if (focusQueue == null) { - focusQueue = queue[i % queue.length]; - } - return obj; - } - - private QueueObject iterateThroughAllQueues(QueueObject obj) throws InterruptedException { - while (obj == null) { - for (ArrayBlockingQueue> queueObjects : queue) { - obj = queueObjects.poll(); - if (obj != null) { - if (focusQueue == null) { - focusQueue = queueObjects; - } - return obj; - } - } - Thread.sleep(10); - } - return obj; - } - - private QueueObject useFocusQueue() throws InterruptedException { - QueueObject obj; - var focusQueue = this.focusQueue; - if (focusQueue != null) { - QueueObject poll = focusQueue.poll(); - if (poll != null) { - obj = poll; - } else { - obj = null; - this.focusQueue = null; - } - } else { - obj = null; - } - return obj; - } - @Override public T next() { if (!hasNext()) { @@ -231,9 +162,7 @@ public void closePipe() { public void closePipe(Throwable e) { if (e != null) { // clear the queue to force the exception - for (ArrayBlockingQueue> queueObjects : queue) { - queueObjects.clear(); - } + queue.clear(); if (e instanceof PipedIteratorException) { this.exception = (PipedIteratorException) e; } else { @@ -241,9 +170,7 @@ public void closePipe(Throwable e) { } } try { - for (ArrayBlockingQueue> queueObjects : queue) { - queueObjects.put(new EndQueueObject()); - } + queue.put(new EndQueueObject()); } catch (InterruptedException ee) { throw new PipedIteratorException("Can't close pipe", ee); } @@ -271,25 +198,9 @@ public Iterator mapWithId(MapIterator.MapWithIdFunction mappingFunc return new MapIterator<>(this, mappingFunction); } - AtomicInteger index = new AtomicInteger(0); - public void addElement(T node) { - int i = Thread.currentThread().hashCode(); - int l = i % queue.length; try { - boolean success = queue[l].offer(new ElementQueueObject(node)); - if (!success) { - focusQueue = queue[l]; - while (!success) { - for (ArrayBlockingQueue> queueObjects : queue) { - success = queueObjects.offer(new ElementQueueObject(node), 1, TimeUnit.MILLISECONDS); - if (success) { - break; - } - } - } - } - + queue.put(new ElementQueueObject(node)); } catch (InterruptedException ee) { throw new PipedIteratorException("Can't add element to pipe", ee); } diff --git a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/iterator/utils/PipedCopyIteratorUnordered.java b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/iterator/utils/PipedCopyIteratorUnordered.java new file mode 100644 index 000000000..798d094e3 --- /dev/null +++ b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/iterator/utils/PipedCopyIteratorUnordered.java @@ -0,0 +1,324 @@ +package com.the_qa_company.qendpoint.core.iterator.utils; + +import java.io.Closeable; +import java.io.IOException; +import java.util.Iterator; +import java.util.Objects; +import java.util.concurrent.ArrayBlockingQueue; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.function.Function; + +/** + * a utility class to create an iterator from the value returned by another + * Thread + * + * @param the iterator type + * @author Antoine Willerval + */ + +public class PipedCopyIteratorUnordered extends PipedCopyIterator { + + /** + * RuntimeException generated by the PipedCopyIterator + * + * @author Antoine Willerval + */ + public static class PipedIteratorException extends RuntimeException { + public PipedIteratorException(String message, Throwable t) { + super(message, t); + } + } + + /** + * Callback for the + * {@link #createOfCallback(PipedCopyIteratorUnordered.PipeCallBack)} method + * + * @param the iterator type + * @author Antoine Willerval + */ + @FunctionalInterface + public interface PipeCallBack { + /** + * method called from the new thread to generate the new data, at the + * end of the callback, the pipe is closed with or without exception + * + * @param pipe the pipe to fill + * @throws Exception any exception returned by the generator + */ + void createPipe(PipedCopyIteratorUnordered pipe) throws Exception; + } + + /** + * create a piped iterator from a callback runner, the call to the callback + * should be made in the callbackRunner + * + * @param callbackRunner the callback runner + * @param type of the iterator + * @return the iterator + */ + public static PipedCopyIteratorUnordered createOfCallback(PipeCallBack callbackRunner) { + PipedCopyIteratorUnordered pipe = new PipedCopyIteratorUnordered<>(); + + Thread thread = new Thread(() -> { + try { + callbackRunner.createPipe(pipe); + pipe.closePipe(); + } catch (Throwable e) { + pipe.closePipe(e); + } + }, "PipeIterator"); + thread.start(); + + // close the thread at end + pipe.attachThread(thread); + + return pipe; + } + + private interface QueueObject { + boolean end(); + + T get(); + } + + private class ElementQueueObject implements QueueObject { + private final T obj; + + private ElementQueueObject(T obj) { + this.obj = obj; + } + + @Override + public boolean end() { + return false; + } + + @Override + public T get() { + return obj; + } + } + + private class EndQueueObject implements QueueObject { + @Override + public boolean end() { + return true; + } + + @Override + public T get() { + throw new IllegalArgumentException(); + } + } + + private final ArrayBlockingQueue>[] queue = new ArrayBlockingQueue[] { + new ArrayBlockingQueue<>(16 * 1024), new ArrayBlockingQueue<>(16 * 1024), + new ArrayBlockingQueue<>(16 * 1024), new ArrayBlockingQueue<>(16 * 1024), + new ArrayBlockingQueue<>(16 * 1024), new ArrayBlockingQueue<>(16 * 1024), + new ArrayBlockingQueue<>(16 * 1024), new ArrayBlockingQueue<>(16 * 1024), + new ArrayBlockingQueue<>(16 * 1024), new ArrayBlockingQueue<>(16 * 1024), + new ArrayBlockingQueue<>(16 * 1024), new ArrayBlockingQueue<>(16 * 1024), + new ArrayBlockingQueue<>(16 * 1024), new ArrayBlockingQueue<>(16 * 1024), + new ArrayBlockingQueue<>(16 * 1024), new ArrayBlockingQueue<>(16 * 1024), + new ArrayBlockingQueue<>(16 * 1024), new ArrayBlockingQueue<>(16 * 1024), + new ArrayBlockingQueue<>(16 * 1024), new ArrayBlockingQueue<>(16 * 1024), + new ArrayBlockingQueue<>(16 * 1024), new ArrayBlockingQueue<>(16 * 1024), + new ArrayBlockingQueue<>(16 * 1024), new ArrayBlockingQueue<>(16 * 1024), }; + + private T next; + private boolean end; + private PipedIteratorException exception; + + private Thread thread; + + AtomicInteger indexHasNext = new AtomicInteger(0); + + volatile ArrayBlockingQueue> focusQueue; + + @Override + public boolean hasNext() { + if (end) { + return false; + } + if (next != null) { + return true; + } + + QueueObject obj; + try { + obj = useFocusQueue(); + + if (obj == null) { + + obj = useThreadBasedQueue(); + } + + } catch (InterruptedException e) { + throw new PipedIteratorException("Can't read pipe", e); + } + + if (obj.end()) { + end = true; + if (exception != null) { + throw exception; + } + return false; + } + next = obj.get(); + return true; + } + + private QueueObject useThreadBasedQueue() throws InterruptedException { + QueueObject obj; + int i = Thread.currentThread().hashCode(); + obj = queue[i % queue.length].poll(); + if (obj == null) { + obj = iterateThroughAllQueues(obj); + } else if (focusQueue == null) { + focusQueue = queue[i % queue.length]; + } + return obj; + } + + private QueueObject iterateThroughAllQueues(QueueObject obj) throws InterruptedException { + while (obj == null) { + for (ArrayBlockingQueue> queueObjects : queue) { + obj = queueObjects.poll(); + if (obj != null) { + if (focusQueue == null) { + focusQueue = queueObjects; + } + return obj; + } + } + Thread.sleep(10); + } + return obj; + } + + private QueueObject useFocusQueue() throws InterruptedException { + QueueObject obj; + var focusQueue = this.focusQueue; + if (focusQueue != null) { + QueueObject poll = focusQueue.poll(); + if (poll != null) { + obj = poll; + } else { + obj = null; + this.focusQueue = null; + } + } else { + obj = null; + } + return obj; + } + + @Override + public T next() { + if (!hasNext()) { + return null; + } + T next = this.next; + this.next = null; + return next; + } + + public void closePipe() { + closePipe(null); + } + + public void closePipe(Throwable e) { + if (e != null) { + // clear the queue to force the exception + for (ArrayBlockingQueue> queueObjects : queue) { + queueObjects.clear(); + } + if (e instanceof PipedIteratorException) { + this.exception = (PipedIteratorException) e; + } else { + this.exception = new PipedIteratorException("closing exception", e); + } + } + try { + for (ArrayBlockingQueue> queueObjects : queue) { + queueObjects.put(new EndQueueObject()); + } + } catch (InterruptedException ee) { + throw new PipedIteratorException("Can't close pipe", ee); + } + } + + /** + * map this iterator to another type + * + * @param mappingFunction the mapping function + * @param the future type + * @return mapped iterator + */ + public Iterator map(Function mappingFunction) { + return new MapIterator<>(this, mappingFunction); + } + + /** + * map this iterator to another type + * + * @param mappingFunction the mapping function + * @param the future type + * @return mapped iterator + */ + public Iterator mapWithId(MapIterator.MapWithIdFunction mappingFunction) { + return new MapIterator<>(this, mappingFunction); + } + + AtomicInteger index = new AtomicInteger(0); + + public void addElement(T node) { + int i = Thread.currentThread().hashCode(); + int l = i % queue.length; + try { + boolean success = queue[l].offer(new ElementQueueObject(node)); + if (!success) { + focusQueue = queue[l]; + while (!success) { + for (ArrayBlockingQueue> queueObjects : queue) { + success = queueObjects.offer(new ElementQueueObject(node), 1, TimeUnit.MILLISECONDS); + if (success) { + break; + } + } + } + } + + } catch (InterruptedException ee) { + throw new PipedIteratorException("Can't add element to pipe", ee); + } + } + + /** + * attach a thread to interrupt with this iterator + * + * @param thread the thread + */ + public void attachThread(Thread thread) { + Objects.requireNonNull(thread, "thread can't be null!"); + if (this.thread != null && this.thread != thread) { + throw new IllegalArgumentException("Thread already attached"); + } + this.thread = thread; + } + + /** + * Allow receiving again elements after an end node + */ + public void reset() { + this.end = false; + } + + @Override + public void close() throws IOException { + if (thread != null) { + thread.interrupt(); + } + } +} diff --git a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/rdf/RDFParserFactory.java b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/rdf/RDFParserFactory.java index 6653451f8..9aaa41bcb 100644 --- a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/rdf/RDFParserFactory.java +++ b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/rdf/RDFParserFactory.java @@ -21,6 +21,7 @@ import com.the_qa_company.qendpoint.core.enums.RDFNotation; import com.the_qa_company.qendpoint.core.exceptions.NotImplementedException; +import com.the_qa_company.qendpoint.core.iterator.utils.PipedCopyIteratorUnordered; import com.the_qa_company.qendpoint.core.options.HDTOptions; import com.the_qa_company.qendpoint.core.options.HDTOptionsKeys; import com.the_qa_company.qendpoint.core.rdf.parsers.RDFDeltaFileParser; @@ -96,8 +97,9 @@ public static RDFParserCallback getParserCallback(RDFNotation notation, HDTOptio */ public static PipedCopyIterator readAsIterator(RDFParserCallback parser, InputStream stream, String baseUri, boolean keepBNode, RDFNotation notation) { - return PipedCopyIterator.createOfCallback(pipe -> parser.doParse(stream, baseUri, notation, keepBNode, - (triple, pos) -> pipe.addElement(triple.tripleToString()))); + return PipedCopyIteratorUnordered + .createOfCallback((PipedCopyIteratorUnordered.PipeCallBack) pipe -> parser.doParse(stream, + baseUri, notation, keepBNode, (triple, pos) -> pipe.addElement(triple.tripleToString()))); } /** @@ -111,8 +113,9 @@ public static PipedCopyIterator readAsIterator(RDFParserCallback p */ public static PipedCopyIterator readAsIterator(RDFParserCallback parser, String file, String baseUri, boolean keepBNode, RDFNotation notation) { - return PipedCopyIterator.createOfCallback(pipe -> parser.doParse(file, baseUri, notation, keepBNode, - (triple, pos) -> pipe.addElement(triple.tripleToString()))); + return PipedCopyIteratorUnordered + .createOfCallback((PipedCopyIteratorUnordered.PipeCallBack) pipe -> parser.doParse(file, + baseUri, notation, keepBNode, (triple, pos) -> pipe.addElement(triple.tripleToString()))); } } From 014985c631a466a9bbc39e6d31cd3795458ca662 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ha=CC=8Avard=20Ottestad?= Date: Wed, 12 Feb 2025 14:41:24 +0100 Subject: [PATCH 11/28] wip --- .../core/util/string/ByteString.java | 66 +++++++++---------- .../core/util/string/ByteStringUtil.java | 18 ++--- 2 files changed, 42 insertions(+), 42 deletions(-) diff --git a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/util/string/ByteString.java b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/util/string/ByteString.java index 9f0bca93e..2a1206f89 100644 --- a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/util/string/ByteString.java +++ b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/util/string/ByteString.java @@ -49,42 +49,10 @@ static ByteString copy(CharSequence csq) { */ byte[] getBuffer(); - @Override - default int compareTo(ByteString other) { - int n = Math.min(length(), other.length()); - int k = 0; - while (k < n) { - char c1 = charAt(k); - char c2 = other.charAt(k); - if (c1 != c2) { - return c1 - c2; - } - k++; - } - return length() - other.length(); - } - -// /* -// * (non-Javadoc) -// * @see java.lang.Comparable#compareTo(java.lang.Object) -// */ // @Override // default int compareTo(ByteString other) { // int n = Math.min(length(), other.length()); -// -// if (n == 0) { -// return length() - other.length(); -// } -// -// byte[] buffer = getBuffer(); -// byte[] buffer1 = other.getBuffer(); -// -// int mismatch = Arrays.mismatch(buffer, buffer1); -// if (mismatch == -1) { -// return length() - other.length(); -// } -// -// int k = mismatch; +// int k = 0; // while (k < n) { // char c1 = charAt(k); // char c2 = other.charAt(k); @@ -96,6 +64,38 @@ default int compareTo(ByteString other) { // return length() - other.length(); // } + /* + * (non-Javadoc) + * @see java.lang.Comparable#compareTo(java.lang.Object) + */ + @Override + default int compareTo(ByteString other) { + int n = Math.min(length(), other.length()); + + if (n == 0) { + return length() - other.length(); + } + + byte[] buffer = getBuffer(); + byte[] buffer1 = other.getBuffer(); + + int mismatch = Arrays.mismatch(buffer, buffer1); + if (mismatch == -1) { + return length() - other.length(); + } + + int k = mismatch; + while (k < n) { + char c1 = charAt(k); + char c2 = other.charAt(k); + if (c1 != c2) { + return c1 - c2; + } + k++; + } + return length() - other.length(); + } + @Override ByteString subSequence(int start, int end); diff --git a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/util/string/ByteStringUtil.java b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/util/string/ByteStringUtil.java index d6231739a..e66d21173 100644 --- a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/util/string/ByteStringUtil.java +++ b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/util/string/ByteStringUtil.java @@ -120,16 +120,16 @@ public static int longestCommonPrefix(CharSequence str1, CharSequence str2) { return longestCommonPrefix(str1, str2, 0); } - public static int longestCommonPrefix(CharSequence str1, CharSequence str2, int from) { - int len = Math.min(str1.length(), str2.length()); - int delta = from; - while (delta < len && str1.charAt(delta) == str2.charAt(delta)) { - delta++; - } - return delta - from; - } +// public static int longestCommonPrefix(CharSequence str1, CharSequence str2, int from) { +// int len = Math.min(str1.length(), str2.length()); +// int delta = from; +// while (delta < len && str1.charAt(delta) == str2.charAt(delta)) { +// delta++; +// } +// return delta - from; +// } - public static int longestCommonPrefixNew(CharSequence str1, CharSequence str2, int from) { + public static int longestCommonPrefix(CharSequence str1, CharSequence str2, int from) { int len = Math.min(str1.length(), str2.length()); if (from >= len) { From f4a2da1ce2e34df47c177fa5adc1831cdadfb0f8 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ha=CC=8Avard=20Ottestad?= Date: Wed, 12 Feb 2025 15:13:52 +0100 Subject: [PATCH 12/28] wip --- .../qendpoint/core/util/crc/CRC.java | 2 ++ .../qendpoint/core/util/crc/CRC16.java | 5 +++++ .../qendpoint/core/util/crc/CRC32.java | 19 +++++++++++++------ .../qendpoint/core/util/crc/CRC8.java | 5 +++++ .../core/util/crc/CRCOutputStream.java | 7 ++++++- .../core/util/string/ByteString.java | 10 ++++++++++ .../core/util/string/ByteStringUtil.java | 8 ++++---- 7 files changed, 45 insertions(+), 11 deletions(-) diff --git a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/util/crc/CRC.java b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/util/crc/CRC.java index 8f7215401..2048517c7 100644 --- a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/util/crc/CRC.java +++ b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/util/crc/CRC.java @@ -63,4 +63,6 @@ public interface CRC extends Comparable { * @return the number of bytes used to store it */ int sizeof(); + + void update8(byte[] b); } diff --git a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/util/crc/CRC16.java b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/util/crc/CRC16.java index d4ccce5b7..59155f027 100644 --- a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/util/crc/CRC16.java +++ b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/util/crc/CRC16.java @@ -141,4 +141,9 @@ public String toString() { public int sizeof() { return 2; } + + @Override + public void update8(byte[] b) { + update(b, 0, 8); + } } diff --git a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/util/crc/CRC32.java b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/util/crc/CRC32.java index 9c79360e6..c2981e251 100644 --- a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/util/crc/CRC32.java +++ b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/util/crc/CRC32.java @@ -53,12 +53,10 @@ public CRC32() { @Override public void update(byte[] buffer, int offset, int length) { - int tbl_idx; - int len = length; int i = offset; while (len-- != 0) { - tbl_idx = (crc32 ^ buffer[i]) & 0xff; + int tbl_idx = (crc32 ^ buffer[i]) & 0xff; crc32 = crc32_table[tbl_idx] ^ (crc32 >>> 8); i++; } @@ -66,12 +64,10 @@ public void update(byte[] buffer, int offset, int length) { @Override public void update(CloseMappedByteBuffer buffer, int offset, int length) { - int tbl_idx; - int len = length; int i = offset; while (len-- != 0) { - tbl_idx = (crc32 ^ buffer.get(i)) & 0xff; + int tbl_idx = (crc32 ^ buffer.get(i)) & 0xff; crc32 = crc32_table[tbl_idx] ^ (crc32 >>> 8); i++; } @@ -133,4 +129,15 @@ public String toString() { public int sizeof() { return 4; } + + public void update8(byte[] buffer) { + crc32 = crc32_table[(crc32 ^ buffer[0]) & 0xff] ^ (crc32 >>> 8); + crc32 = crc32_table[(crc32 ^ buffer[1]) & 0xff] ^ (crc32 >>> 8); + crc32 = crc32_table[(crc32 ^ buffer[2]) & 0xff] ^ (crc32 >>> 8); + crc32 = crc32_table[(crc32 ^ buffer[3]) & 0xff] ^ (crc32 >>> 8); + crc32 = crc32_table[(crc32 ^ buffer[4]) & 0xff] ^ (crc32 >>> 8); + crc32 = crc32_table[(crc32 ^ buffer[5]) & 0xff] ^ (crc32 >>> 8); + crc32 = crc32_table[(crc32 ^ buffer[6]) & 0xff] ^ (crc32 >>> 8); + crc32 = crc32_table[(crc32 ^ buffer[7]) & 0xff] ^ (crc32 >>> 8); + } } diff --git a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/util/crc/CRC8.java b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/util/crc/CRC8.java index e616c7b90..4964e20aa 100644 --- a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/util/crc/CRC8.java +++ b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/util/crc/CRC8.java @@ -141,4 +141,9 @@ public String toString() { public int sizeof() { return 1; } + + @Override + public void update8(byte[] b) { + update(b, 0, 8); + } } diff --git a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/util/crc/CRCOutputStream.java b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/util/crc/CRCOutputStream.java index e5a0ef128..bfb7f8f54 100644 --- a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/util/crc/CRCOutputStream.java +++ b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/util/crc/CRCOutputStream.java @@ -57,7 +57,12 @@ public void write(byte[] b) throws IOException { @Override public void write(byte[] b, int off, int len) throws IOException { - crc.update(b, off, len); + if (len == 8 && off == 0) { + crc.update8(b); + } else { + crc.update(b, off, len); + } + out.write(b, off, len); } diff --git a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/util/string/ByteString.java b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/util/string/ByteString.java index 2a1206f89..7c8998b81 100644 --- a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/util/string/ByteString.java +++ b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/util/string/ByteString.java @@ -76,6 +76,12 @@ default int compareTo(ByteString other) { return length() - other.length(); } + char cc1 = charAt(0); + char cc2 = other.charAt(0); + if (cc1 != cc2) { + return cc1 - cc2; + } + byte[] buffer = getBuffer(); byte[] buffer1 = other.getBuffer(); @@ -84,6 +90,10 @@ default int compareTo(ByteString other) { return length() - other.length(); } + return extracted(other, mismatch, n); + } + + private int extracted(ByteString other, int mismatch, int n) { int k = mismatch; while (k < n) { char c1 = charAt(k); diff --git a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/util/string/ByteStringUtil.java b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/util/string/ByteStringUtil.java index e66d21173..8c498d687 100644 --- a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/util/string/ByteStringUtil.java +++ b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/util/string/ByteStringUtil.java @@ -135,10 +135,10 @@ public static int longestCommonPrefix(CharSequence str1, CharSequence str2, int if (from >= len) { return 0; } -// -// if (str1.charAt(from) != str2.charAt(from)) { -// return 0; -// } + + if (str1.charAt(from) != str2.charAt(from)) { + return 0; + } if (str1 instanceof ByteString && str2 instanceof ByteString) { From b9665c58a20b35cda891a9b6fdc795454ab8c7da Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ha=CC=8Avard=20Ottestad?= Date: Wed, 12 Feb 2025 21:31:42 +0100 Subject: [PATCH 13/28] wip --- .../core/util/string/ByteString.java | 52 ++++++-- .../core/util/string/ByteStringUtil.java | 112 +++++++++++++++++- 2 files changed, 150 insertions(+), 14 deletions(-) diff --git a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/util/string/ByteString.java b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/util/string/ByteString.java index 7c8998b81..104247955 100644 --- a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/util/string/ByteString.java +++ b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/util/string/ByteString.java @@ -1,5 +1,6 @@ package com.the_qa_company.qendpoint.core.util.string; +import jdk.incubator.vector.ByteVector; import jdk.incubator.vector.ShortVector; import jdk.incubator.vector.VectorMask; import jdk.incubator.vector.VectorOperators; @@ -76,21 +77,54 @@ default int compareTo(ByteString other) { return length() - other.length(); } - char cc1 = charAt(0); - char cc2 = other.charAt(0); - if (cc1 != cc2) { - return cc1 - cc2; - } - byte[] buffer = getBuffer(); byte[] buffer1 = other.getBuffer(); - int mismatch = Arrays.mismatch(buffer, buffer1); - if (mismatch == -1) { + if (n < 128) { + return naive(other, n, buffer, buffer1); + } + + return vector(other, n, buffer, buffer1); + + } + + private int vector(ByteString other, int n, byte[] buffer, byte[] buffer1) { + int mismatch = mismatchVectorByte(buffer, buffer1); + if (mismatch == -1 || mismatch >= n) { return length() - other.length(); } + return charAt(mismatch) - other.charAt(mismatch); + } - return extracted(other, mismatch, n); + private int naive(ByteString other, int n, byte[] buffer, byte[] buffer1) { + for (int i = 0; i < 32 && i < n; i++) { + if (buffer[i] != buffer1[i]) { + return charAt(i) - other.charAt(i); + } + } + return length() - other.length(); + } + + default int mismatchVectorByte(byte[] byteData1, byte[] byteData2) { + int length = Math.min(byteData1.length, byteData2.length); + int index = 0; + for (; index < ByteVector.SPECIES_PREFERRED.loopBound(length); index += ByteVector.SPECIES_PREFERRED.length()) { + ByteVector vector1 = ByteVector.fromArray(ByteVector.SPECIES_PREFERRED, byteData1, index); + ByteVector vector2 = ByteVector.fromArray(ByteVector.SPECIES_PREFERRED, byteData2, index); + VectorMask mask = vector1.compare(VectorOperators.NE, vector2); + if (mask.anyTrue()) { + return index + mask.firstTrue(); + } + } + // process the tail + int mismatch = -1; + for (int i = index; i < length; ++i) { + if (byteData1[i] != byteData2[i]) { + mismatch = i; + break; + } + } + return mismatch; } private int extracted(ByteString other, int mismatch, int n) { diff --git a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/util/string/ByteStringUtil.java b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/util/string/ByteStringUtil.java index 8c498d687..f29d8e4b3 100644 --- a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/util/string/ByteStringUtil.java +++ b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/util/string/ByteStringUtil.java @@ -23,11 +23,17 @@ import java.io.OutputStream; import java.nio.ByteBuffer; import java.nio.charset.Charset; +import java.util.ArrayList; import java.util.Arrays; +import java.util.Random; import com.the_qa_company.qendpoint.core.exceptions.NotImplementedException; import com.the_qa_company.qendpoint.core.util.io.BigByteBuffer; import com.the_qa_company.qendpoint.core.util.io.BigMappedByteBuffer; +import jdk.incubator.vector.ByteVector; +import jdk.incubator.vector.VectorMask; +import jdk.incubator.vector.VectorOperators; +import org.apache.jena.base.Sys; import static java.nio.charset.StandardCharsets.UTF_8; @@ -129,6 +135,56 @@ public static int longestCommonPrefix(CharSequence str1, CharSequence str2) { // return delta - from; // } + public static void main(String[] args) { + ArrayList bytes1 = new ArrayList<>(); + ArrayList bytes2 = new ArrayList<>(); + + Random random = new Random(); + + for (int i = 0; i < 2048 * 32; i++) { + byte[] byteArray = new byte[random.nextInt(2048)]; + for (int j = 0; j < byteArray.length; j++) { + byteArray[j] = (byte) random.nextInt(); + } + bytes1.add(byteArray); + } + + for (int i = 0; i < 2048 * 32; i++) { + byte[] byteArray = new byte[random.nextInt(2048)]; + for (int j = 0; j < byteArray.length; j++) { + byteArray[j] = (byte) random.nextInt(); + } + bytes2.add(byteArray); + } + + int[] millis = new int[2048]; + int[] mismatch = new int[2048]; + + for (int k = 0; k < 10; k++) { + for (int i = 0; i < bytes1.size(); i++) { + byte[] byteArray1 = bytes1.get(i); + byte[] byteArray2 = bytes2.get(i); + + long start = System.nanoTime(); + int mismatch1 = Arrays.mismatch(byteArray1, byteArray2); + long l = System.nanoTime(); + if (millis[Math.min(byteArray1.length, byteArray2.length)] == 0) { + millis[Math.min(byteArray1.length, byteArray2.length)] = (int) (l - start); + mismatch[Math.min(byteArray1.length, byteArray2.length)] = mismatch1; + } else { + millis[Math.min(byteArray1.length, byteArray2.length)] = Math + .min(millis[Math.min(byteArray1.length, byteArray2.length)], (int) (l - start)); + mismatch[Math.min(byteArray1.length, byteArray2.length)] = mismatch1; + } + } + } + + for (int i = 0; i < millis.length; i++) { + System.out.println(i + " " + millis[i] + " ns " + mismatch[i]); + } + + } + public static int longestCommonPrefix(CharSequence str1, CharSequence str2, int from) { int len = Math.min(str1.length(), str2.length()); @@ -142,14 +198,17 @@ public static int longestCommonPrefix(CharSequence str1, CharSequence str2, int if (str1 instanceof ByteString && str2 instanceof ByteString) { + if (len - from < 128) { + return naive(str1, str2, from, len); + } + byte[] buffer = ((ByteString) str1).getBuffer(); byte[] buffer2 = ((ByteString) str2).getBuffer(); // System.out.println("mismatch: " + i); - int missmatch = Arrays.mismatch(buffer, from, len, buffer2, from, len); - if (missmatch == -1) { - return len - from; + if (from == 0) { + return vector(buffer, buffer2, len); } else { - return missmatch; + return mismatch(from, buffer, len, buffer2); } // int delta = from; @@ -171,14 +230,57 @@ public static int longestCommonPrefix(CharSequence str1, CharSequence str2, int // return missmatch - from; } + return naive(str1, str2, from, len); + } + + private static int mismatch(int from, byte[] buffer, int len, byte[] buffer2) { + int missmatch = Arrays.mismatch(buffer, from, len, buffer2, from, len); + if (missmatch == -1) { + return len - from; + } else { + return missmatch; + } + } + + private static int vector(byte[] buffer, byte[] buffer2, int len) { + int mismatch = mismatchVectorByte(buffer, buffer2); + if (mismatch == -1 || mismatch >= len) { + return len; + } else { + return mismatch; + } + } + + private static int naive(CharSequence str1, CharSequence str2, int from, int len) { int delta = from; while (delta < len && str1.charAt(delta) == str2.charAt(delta)) { delta++; } - // System.out.println("i: " + i); return delta - from; } + private static int mismatchVectorByte(byte[] byteData1, byte[] byteData2) { + int length = Math.min(byteData1.length, byteData2.length); + int index = 0; + for (; index < ByteVector.SPECIES_PREFERRED.loopBound(length); index += ByteVector.SPECIES_PREFERRED.length()) { + ByteVector vector1 = ByteVector.fromArray(ByteVector.SPECIES_PREFERRED, byteData1, index); + ByteVector vector2 = ByteVector.fromArray(ByteVector.SPECIES_PREFERRED, byteData2, index); + VectorMask mask = vector1.compare(VectorOperators.NE, vector2); + if (mask.anyTrue()) { + return index + mask.firstTrue(); + } + } + // process the tail + int mismatch = -1; + for (int i = index; i < length; ++i) { + if (byteData1[i] != byteData2[i]) { + mismatch = i; + break; + } + } + return mismatch; + } + public static int strcmp(CharSequence str, byte[] buff2, int off2) { byte[] buff1; int off1; From 2e4138d325af07642b234efbd3b5e4eceb11217b Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ha=CC=8Avard=20Ottestad?= Date: Wed, 12 Feb 2025 21:33:25 +0100 Subject: [PATCH 14/28] wip --- .../qendpoint/core/util/string/ByteString.java | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/util/string/ByteString.java b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/util/string/ByteString.java index 104247955..959c6b8cb 100644 --- a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/util/string/ByteString.java +++ b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/util/string/ByteString.java @@ -77,10 +77,15 @@ default int compareTo(ByteString other) { return length() - other.length(); } + int i = charAt(0) - other.charAt(0); + if (i != 0) { + return i; + } + byte[] buffer = getBuffer(); byte[] buffer1 = other.getBuffer(); - if (n < 128) { + if (n < 16) { return naive(other, n, buffer, buffer1); } From d17961fe864ba875edae58c688d89f345053ea74 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ha=CC=8Avard=20Ottestad?= Date: Wed, 12 Feb 2025 22:43:24 +0100 Subject: [PATCH 15/28] wip --- .../qendpoint/core/util/string/ByteString.java | 14 +++++++------- .../qendpoint/core/util/string/ByteStringUtil.java | 12 ++++++------ 2 files changed, 13 insertions(+), 13 deletions(-) diff --git a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/util/string/ByteString.java b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/util/string/ByteString.java index 959c6b8cb..2f60f3350 100644 --- a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/util/string/ByteString.java +++ b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/util/string/ByteString.java @@ -77,24 +77,24 @@ default int compareTo(ByteString other) { return length() - other.length(); } - int i = charAt(0) - other.charAt(0); - if (i != 0) { - return i; - } - byte[] buffer = getBuffer(); byte[] buffer1 = other.getBuffer(); - if (n < 16) { + if (n < 64) { return naive(other, n, buffer, buffer1); } + int i = charAt(0) - other.charAt(0); + if (i != 0) { + return i; + } + return vector(other, n, buffer, buffer1); } private int vector(ByteString other, int n, byte[] buffer, byte[] buffer1) { - int mismatch = mismatchVectorByte(buffer, buffer1); + int mismatch = Arrays.mismatch(buffer, buffer1); if (mismatch == -1 || mismatch >= n) { return length() - other.length(); } diff --git a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/util/string/ByteStringUtil.java b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/util/string/ByteStringUtil.java index f29d8e4b3..68e2dfb4a 100644 --- a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/util/string/ByteStringUtil.java +++ b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/util/string/ByteStringUtil.java @@ -198,18 +198,18 @@ public static int longestCommonPrefix(CharSequence str1, CharSequence str2, int if (str1 instanceof ByteString && str2 instanceof ByteString) { - if (len - from < 128) { + if (len - from < 64) { return naive(str1, str2, from, len); } byte[] buffer = ((ByteString) str1).getBuffer(); byte[] buffer2 = ((ByteString) str2).getBuffer(); // System.out.println("mismatch: " + i); - if (from == 0) { - return vector(buffer, buffer2, len); - } else { - return mismatch(from, buffer, len, buffer2); - } +// if (from == 0) { +// return vector(buffer, buffer2, len); +// } else { + return mismatch(from, buffer, len, buffer2); +// } // int delta = from; // while (delta < len && str1.charAt(delta) == str2.charAt(delta)) { From 69fade7c1d0378650ac7665dbe875d8da399085a Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ha=CC=8Avard=20Ottestad?= Date: Fri, 14 Feb 2025 11:47:26 +0100 Subject: [PATCH 16/28] parallel merging --- pom.xml | 8 +- qendpoint-backend/pom.xml | 2 + qendpoint-core/pom.xml | 6 +- .../utils/MergeExceptionIterator.java | 212 ++++++++++++++---- qendpoint-store/pom.xml | 2 + 5 files changed, 178 insertions(+), 52 deletions(-) diff --git a/pom.xml b/pom.xml index e4845c971..ab40b6ff1 100644 --- a/pom.xml +++ b/pom.xml @@ -63,10 +63,10 @@ UTF-8 - 17 - 17 - 17 - 17 + 21 + 21 + 21 + 21 diff --git a/qendpoint-backend/pom.xml b/qendpoint-backend/pom.xml index 8cac3f652..c4affaa42 100644 --- a/qendpoint-backend/pom.xml +++ b/qendpoint-backend/pom.xml @@ -147,6 +147,8 @@ ${java.source.version} ${java.target.version} + + --add-exports java.base/jdk.internal.misc=ALL-UNNAMED --add-exports diff --git a/qendpoint-core/pom.xml b/qendpoint-core/pom.xml index 19fc512ab..d0adbad4a 100644 --- a/qendpoint-core/pom.xml +++ b/qendpoint-core/pom.xml @@ -17,9 +17,11 @@ org.apache.maven.plugins maven-compiler-plugin - 17 - 17 + 21 + 21 + + --add-exports java.base/jdk.internal.misc=ALL-UNNAMED --add-exports diff --git a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/iterator/utils/MergeExceptionIterator.java b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/iterator/utils/MergeExceptionIterator.java index fbf7bf933..0cb17afb4 100644 --- a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/iterator/utils/MergeExceptionIterator.java +++ b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/iterator/utils/MergeExceptionIterator.java @@ -1,8 +1,14 @@ package com.the_qa_company.qendpoint.core.iterator.utils; +import java.util.ArrayDeque; import java.util.Arrays; import java.util.Comparator; +import java.util.Deque; import java.util.List; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.Executor; +import java.util.concurrent.Executors; import java.util.function.BiFunction; import java.util.function.Function; @@ -144,8 +150,25 @@ public static ExceptionIterator buildOfTree( buildOfTree(itFunction, comp, array, mid, end), comp); } - private final ExceptionIterator in1, in2; + private final ExceptionIterator in1; + private final ExceptionIterator in2; private final Comparator comp; + private final int chunkSize = 1024 * 4; + private final Executor executor = Executors.newVirtualThreadPerTaskExecutor(); // Could + // be + // a + // ForkJoinPool.commonPool(), + // or + // a + // custom + // pool + + private final Deque chunk1 = new ArrayDeque<>(); + private final Deque chunk2 = new ArrayDeque<>(); + + // Local buffer to store merged chunks + private final Deque buffer = new ArrayDeque<>(); + private T next; private T prevE1; private T prevE2; @@ -158,47 +181,21 @@ public MergeExceptionIterator(ExceptionIterator in1, ExceptionIterator> future1 = chunk1.size() < chunkSize && in1.hasNext() + ? fetchChunkAsync(in1, chunk1, chunkSize) + : null; + CompletableFuture> future2 = chunk2.size() < chunkSize && in2.hasNext() + ? fetchChunkAsync(in2, chunk2, chunkSize) + : null; + + // Wait for both tasks to complete + if (future1 != null && future2 != null) { + CompletableFuture.allOf(future1, future2).join(); + } + + Deque chunk1; + Deque chunk2; + try { + chunk1 = future1 != null ? future1.get() : this.chunk1; + chunk2 = future2 != null ? future2.get() : this.chunk2; + } catch (InterruptedException ie) { + // Restore interrupt status + Thread.currentThread().interrupt(); + throw new RuntimeException("Interrupted while fetching chunks in parallel", ie); + } catch (ExecutionException ee) { + // If our underlying fetch threw a checked exception E, unwrap and + // throw it + Throwable cause = ee.getCause(); + if (cause instanceof Exception ex) { + // You may need a different mechanism to re-throw the correct + // type E + // e.g. reflection or a wrapper if E is known + throw asE(ex); + } else { + throw new RuntimeException("Unexpected error in parallel fetch", cause); + } + } + if (chunk1.isEmpty()) { + while (!chunk2.isEmpty()) { + buffer.addLast(chunk2.pollFirst()); + } + } else if (chunk2.isEmpty()) { + while (!chunk1.isEmpty()) { + buffer.addLast(chunk1.pollFirst()); + } + } else { + // Merge the two fetched chunks in sorted order + mergeChunksIntoBuffer(chunk1, chunk2); } - T next = this.next; - this.next = null; - return next; + } + + /** + * Helper to schedule a chunk fetch on the given iterator and return a + * CompletableFuture. Because T can throw a checked exception E, we wrap the + * call and handle exceptions carefully. + */ + private CompletableFuture> fetchChunkAsync(ExceptionIterator iter, Deque chunk, int n) { + CompletableFuture> future = new CompletableFuture<>(); +// executor.execute(() -> { + try { + Deque result = fetchChunk(iter, chunk, n); + future.complete(result); + } catch (Exception e) { + future.completeExceptionally(e); + } +// }); + return future; + } + + /** + * Actual synchronous fetch of up to 'n' items from the child iterator. + */ + private Deque fetchChunk(ExceptionIterator iter, Deque list, int n) throws E { + while (list.size() < n && iter.hasNext()) { + list.addLast(iter.next()); + } + return list; + } + + /** + * Merge two sorted lists into our buffer in ascending order. If the child + * iterators are guaranteed sorted, you can do this linear merge. Otherwise, + * you'd need a custom approach (possibly sorting the partial chunks). + */ + private void mergeChunksIntoBuffer(Deque c1, Deque c2) { + + if (c1.isEmpty() || c2.isEmpty()) { + return; + } + + // this assumes that each of the two chunks is sorted + T c1First = c1.peek(); + T c2Last = c2.peekLast(); + if (comp.compare(c1First, c2Last) > 0) { + buffer.addAll(c2); + c2.clear(); + return; + } + + T c2First = c2.peek(); + T c1Last = c1.peekLast(); + if (comp.compare(c2First, c1Last) > 0) { + buffer.addAll(c1); + c1.clear(); + return; + } + + while (!(c1.isEmpty() || c2.isEmpty())) { + if (comp.compare(c1.peek(), c2.peek()) < 0) { + buffer.addLast(c1.pollFirst()); + } else { + buffer.addLast(c2.pollFirst()); + } + } + } + + /** + * Utility to cast a generic Exception to E if needed, or wrap as + * RuntimeException. Adjust as necessary for your real-world scenario. + */ + @SuppressWarnings("unchecked") + private E asE(Exception ex) { + return (E) ex; } } diff --git a/qendpoint-store/pom.xml b/qendpoint-store/pom.xml index 9053c9843..dfdb5b776 100644 --- a/qendpoint-store/pom.xml +++ b/qendpoint-store/pom.xml @@ -184,6 +184,8 @@ ${java.source.version} ${java.target.version} + + --add-exports java.base/jdk.internal.misc=ALL-UNNAMED --add-exports From 6c534fe715c364acfeb1ca5b566090774606191a Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ha=CC=8Avard=20Ottestad?= Date: Tue, 18 Feb 2025 13:21:38 +0100 Subject: [PATCH 17/28] wip --- .../impl/diskimport/SectionCompressor.java | 2 +- .../utils/MergeExceptionIterator.java | 221 +++------- .../utils/MergeExceptionParallelIterator.java | 332 +++++++++++++++ .../core/iterator/utils/ParallelMerge.java | 378 ++++++++++++++++++ 4 files changed, 767 insertions(+), 166 deletions(-) create mode 100644 qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/iterator/utils/MergeExceptionParallelIterator.java create mode 100644 qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/iterator/utils/ParallelMerge.java diff --git a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/hdt/impl/diskimport/SectionCompressor.java b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/hdt/impl/diskimport/SectionCompressor.java index 09e208473..a183821b8 100644 --- a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/hdt/impl/diskimport/SectionCompressor.java +++ b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/hdt/impl/diskimport/SectionCompressor.java @@ -60,7 +60,7 @@ public SectionCompressor(CloseSuppressPath baseFileName, AsyncIteratorFetcher ExceptionIterator buildOfTree( buildOfTree(itFunction, comp, array, mid, end), comp); } - private final ExceptionIterator in1; - private final ExceptionIterator in2; + private final ExceptionIterator in1, in2; private final Comparator comp; - private final int chunkSize = 1024 * 4; - private final Executor executor = Executors.newVirtualThreadPerTaskExecutor(); // Could - // be - // a - // ForkJoinPool.commonPool(), - // or - // a - // custom - // pool - - private final Deque chunk1 = new ArrayDeque<>(); - private final Deque chunk2 = new ArrayDeque<>(); - - // Local buffer to store merged chunks - private final Deque buffer = new ArrayDeque<>(); - private T next; private T prevE1; private T prevE2; @@ -177,25 +155,56 @@ public MergeExceptionIterator(ExceptionIterator in1, ExceptionIterator> future1 = chunk1.size() < chunkSize && in1.hasNext() - ? fetchChunkAsync(in1, chunk1, chunkSize) - : null; - CompletableFuture> future2 = chunk2.size() < chunkSize && in2.hasNext() - ? fetchChunkAsync(in2, chunk2, chunkSize) - : null; - - // Wait for both tasks to complete - if (future1 != null && future2 != null) { - CompletableFuture.allOf(future1, future2).join(); - } - - Deque chunk1; - Deque chunk2; - try { - chunk1 = future1 != null ? future1.get() : this.chunk1; - chunk2 = future2 != null ? future2.get() : this.chunk2; - } catch (InterruptedException ie) { - // Restore interrupt status - Thread.currentThread().interrupt(); - throw new RuntimeException("Interrupted while fetching chunks in parallel", ie); - } catch (ExecutionException ee) { - // If our underlying fetch threw a checked exception E, unwrap and - // throw it - Throwable cause = ee.getCause(); - if (cause instanceof Exception ex) { - // You may need a different mechanism to re-throw the correct - // type E - // e.g. reflection or a wrapper if E is known - throw asE(ex); - } else { - throw new RuntimeException("Unexpected error in parallel fetch", cause); - } - } - if (chunk1.isEmpty()) { - while (!chunk2.isEmpty()) { - buffer.addLast(chunk2.pollFirst()); - } - } else if (chunk2.isEmpty()) { - while (!chunk1.isEmpty()) { - buffer.addLast(chunk1.pollFirst()); - } - } else { - // Merge the two fetched chunks in sorted order - mergeChunksIntoBuffer(chunk1, chunk2); - } - } - - /** - * Helper to schedule a chunk fetch on the given iterator and return a - * CompletableFuture. Because T can throw a checked exception E, we wrap the - * call and handle exceptions carefully. - */ - private CompletableFuture> fetchChunkAsync(ExceptionIterator iter, Deque chunk, int n) { - CompletableFuture> future = new CompletableFuture<>(); -// executor.execute(() -> { + @Override + public T next() throws E { try { - Deque result = fetchChunk(iter, chunk, n); - future.complete(result); - } catch (Exception e) { - future.completeExceptionally(e); - } -// }); - return future; - } - - /** - * Actual synchronous fetch of up to 'n' items from the child iterator. - */ - private Deque fetchChunk(ExceptionIterator iter, Deque list, int n) throws E { - while (list.size() < n && iter.hasNext()) { - list.addLast(iter.next()); - } - return list; - } - - /** - * Merge two sorted lists into our buffer in ascending order. If the child - * iterators are guaranteed sorted, you can do this linear merge. Otherwise, - * you'd need a custom approach (possibly sorting the partial chunks). - */ - private void mergeChunksIntoBuffer(Deque c1, Deque c2) { - - if (c1.isEmpty() || c2.isEmpty()) { - return; - } - - // this assumes that each of the two chunks is sorted - T c1First = c1.peek(); - T c2Last = c2.peekLast(); - if (comp.compare(c1First, c2Last) > 0) { - buffer.addAll(c2); - c2.clear(); - return; - } - T c2First = c2.peek(); - T c1Last = c1.peekLast(); - if (comp.compare(c2First, c1Last) > 0) { - buffer.addAll(c1); - c1.clear(); - return; - } - - while (!(c1.isEmpty() || c2.isEmpty())) { - if (comp.compare(c1.peek(), c2.peek()) < 0) { - buffer.addLast(c1.pollFirst()); - } else { - buffer.addLast(c2.pollFirst()); + if (!hasNext()) { + return null; } + T next = this.next; + this.next = null; + return next; + } catch (Exception e) { + throw (E) e; } } - - /** - * Utility to cast a generic Exception to E if needed, or wrap as - * RuntimeException. Adjust as necessary for your real-world scenario. - */ - @SuppressWarnings("unchecked") - private E asE(Exception ex) { - return (E) ex; - } } diff --git a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/iterator/utils/MergeExceptionParallelIterator.java b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/iterator/utils/MergeExceptionParallelIterator.java new file mode 100644 index 000000000..0426cea9e --- /dev/null +++ b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/iterator/utils/MergeExceptionParallelIterator.java @@ -0,0 +1,332 @@ +package com.the_qa_company.qendpoint.core.iterator.utils; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.ArrayDeque; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Comparator; +import java.util.Deque; +import java.util.List; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CompletionException; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.Executor; +import java.util.concurrent.Executors; +import java.util.function.BiFunction; +import java.util.function.Function; + +public class MergeExceptionParallelIterator implements ExceptionIterator { + + private static final Logger log = LoggerFactory.getLogger(MergeExceptionParallelIterator.class); + + /** + * Create a tree of merge iterators from an array of element + * + * @param itFunction a function to create an iterator from an element + * @param comp comparator for the merge iterator + * @param array the elements + * @param length the number of elements + * @param input of the element + * @param type of the element in the iterator + * @param exception returned by the iterator + * @return the iterator + */ + public static ExceptionIterator buildOfTree( + Function> itFunction, Comparator comp, I[] array, int length) { + return buildOfTree(itFunction, comp, array, 0, length); + } + + /** + * Create a tree of merge iterators from an array of element + * + * @param itFunction a function to create an iterator from an element + * @param comp comparator for the merge iterator + * @param array the elements + * @param start the start of the array (inclusive) + * @param end the end of the array (exclusive) + * @param type of the element + * @param exception returned by the iterator + * @return the iterator + */ + public static ExceptionIterator buildOfTree( + Function> itFunction, Comparator comp, I[] array, int start, int end) { + return buildOfTree(itFunction, comp, Arrays.asList(array), start, end); + } + + /** + * Create a tree of merge iterators from an array of element + * + * @param itFunction a function to create an iterator from an element + * @param comp comparator for the merge iterator + * @param array the elements + * @param start the start of the array (inclusive) + * @param end the end of the array (exclusive) + * @param type of the element + * @param exception returned by the iterator + * @return the iterator + */ + public static ExceptionIterator buildOfTree( + Function> itFunction, Comparator comp, List array, int start, int end) { + return buildOfTree((index, o) -> itFunction.apply(o), comp, array, start, end); + } + + /** + * Create a tree of merge iterators from an array of element + * + * @param itFunction a function to create an iterator from an element + * @param array the elements + * @param start the start of the array (inclusive) + * @param end the end of the array (exclusive) + * @param type of the element + * @param exception returned by the iterator + * @return the iterator + */ + public static , E extends Exception> ExceptionIterator buildOfTree( + Function> itFunction, List array, int start, int end) { + return buildOfTree((index, o) -> itFunction.apply(o), Comparable::compareTo, array, start, end); + } + + /** + * Create a tree of merge iterators from an array of element + * + * @param array the elements + * @param start the start of the array (inclusive) + * @param end the end of the array (exclusive) + * @param type of the element + * @param exception returned by the iterator + * @return the iterator + */ + public static , E extends Exception> ExceptionIterator buildOfTree( + List> array, int start, int end) { + return buildOfTree(Function.identity(), Comparable::compareTo, array, start, end); + } + + /** + * Create a tree of merge iterators from an array of element + * + * @param array the elements + * @param type of the element + * @param exception returned by the iterator + * @return the iterator + */ + public static , E extends Exception> ExceptionIterator buildOfTree( + List> array) { + return MergeExceptionParallelIterator.buildOfTree(Function.identity(), Comparable::compareTo, array, 0, + array.size()); + } + + /** + * Create a tree of merge iterators from an array of element + * + * @param array the elements + * @param comparator comparator for the merge iterator + * @param type of the element + * @param exception returned by the iterator + * @return the iterator + */ + public static ExceptionIterator buildOfTree(List> array, + Comparator comparator) { + return buildOfTree(Function.identity(), comparator, array, 0, array.size()); + } + + /** + * Create a tree of merge iterators from an array of element + * + * @param itFunction a function to create an iterator from an element + * @param comp comparator for the merge iterator + * @param array the elements + * @param start the start of the array (inclusive) + * @param end the end of the array (exclusive) + * @param type of the element + * @param exception returned by the iterator + * @return the iterator + */ + public static ExceptionIterator buildOfTree( + BiFunction> itFunction, Comparator comp, List array, int start, + int end) { + int length = end - start; + if (length <= 0) { + return ExceptionIterator.empty(); + } + if (length == 1) { + return itFunction.apply(start, array.get(start)); + } + int mid = (start + end) / 2; + return new MergeExceptionParallelIterator<>(buildOfTree(itFunction, comp, array, start, mid), + buildOfTree(itFunction, comp, array, mid, end), comp); + } + + private final ExceptionIterator in1; + private final ExceptionIterator in2; + private final Comparator comp; + private final int chunkSize = 4096; + private final Executor executor = Executors.newVirtualThreadPerTaskExecutor(); + + // Each child's buffered items (at most chunkSize). We'll treat these like + // queues. + private final Deque buffer1 = new ArrayDeque<>(); + private final Deque buffer2 = new ArrayDeque<>(); + + // Futures for the next chunk fetch (if currently in progress) + private CompletableFuture> future1 = null; + private CompletableFuture> future2 = null; + + public MergeExceptionParallelIterator(ExceptionIterator in1, ExceptionIterator in2, + Comparator comp) { + this.in1 = in1; + this.in2 = in2; + this.comp = comp; + } + + @Override + public boolean hasNext() throws E { + // Attempt to ensure we have at least one item available + prepareNextItem(); + // If both buffers are empty now, we really have no more data + return !(buffer1.isEmpty() && buffer2.isEmpty()); + } + + @Override + public T next() throws E { + if (!hasNext()) { + return null; // or throw NoSuchElementException + } + // We know there's at least one item in buffer1 or buffer2 + T result; + if (buffer1.isEmpty()) { + // Must come from buffer2 + result = buffer2.pollFirst(); + } else if (buffer2.isEmpty()) { + // Must come from buffer1 + result = buffer1.pollFirst(); + } else { + // Compare the heads + T head1 = buffer1.peekFirst(); + T head2 = buffer2.peekFirst(); + if (comp.compare(head1, head2) <= 0) { + result = buffer1.pollFirst(); + } else { + result = buffer2.pollFirst(); + } + } + return result; + } + + @Override + public long getSize() { + long s1 = in1.getSize(); + long s2 = in2.getSize(); + if (s1 == -1 || s2 == -1) { + return -1; + } + return s1 + s2; + } + + /** + * Ensures at least one buffer is non-empty if data remains. If both are + * empty, we fetch from both children in parallel. + */ + private void prepareNextItem() throws E { + // If both buffers are already non-empty, nothing to do + if (!buffer1.isEmpty() || !buffer2.isEmpty()) { + return; + } + + // We may need to start or finish a fetch for each child: + boolean need1 = buffer1.isEmpty() && in1.hasNext(); + boolean need2 = buffer2.isEmpty() && in2.hasNext(); + +// if (need1 && !need2) { +// if (buffer2.size() < chunkSize / 2 && in2.hasNext()) { +// need2 = true; +// } +// } +// if (need2 && !need1) { +// if (buffer1.size() < chunkSize / 2 && in1.hasNext()) { +// need1 = true; +// } +// } + + // If buffer1 is empty and child1 has data, ensure we have a future + if (need1 && future1 == null) { + future1 = fetchChunkAsync(in1, chunkSize); + } + // If buffer2 is empty and child2 has data, ensure we have a future + if (need2 && future2 == null) { + future2 = fetchChunkAsync(in2, chunkSize); + } + + // If we started any future(s), wait for them all at once + if (future1 != null || future2 != null) { + CompletableFuture f1 = (future1 != null) ? future1 : CompletableFuture.completedFuture(null); + CompletableFuture f2 = (future2 != null) ? future2 : CompletableFuture.completedFuture(null); + + // Wait for both to complete (parallel fetch) + CompletableFuture.allOf(f1, f2).join(); + + // Drain each completed future into its buffer + if (future1 != null) { + addToBuffer(future1, buffer1); + future1 = null; + } + if (future2 != null) { + addToBuffer(future2, buffer2); + future2 = null; + } + } + } + + /** + * Helper to move the fetched chunk from a completed future into the buffer. + * Handles exceptions properly. + */ + private void addToBuffer(CompletableFuture> future, Deque buffer) throws E { + List chunk; + try { + chunk = future.get(); // already done, so non-blocking + } catch (InterruptedException ie) { + Thread.currentThread().interrupt(); + throw new RuntimeException("Interrupted while fetching chunk", ie); + } catch (ExecutionException ee) { + Throwable cause = ee.getCause(); + if (cause instanceof Exception ex) { + throw asE(ex); + } else { + throw new RuntimeException("Error in parallel chunk fetch", cause); + } + } + chunk.forEach(buffer::addLast); + } + + /** + * Asynchronously fetch up to 'n' items from 'iter' on the executor. + */ + private CompletableFuture> fetchChunkAsync(ExceptionIterator iter, int n) { + return CompletableFuture.supplyAsync(() -> { + try { + return fetchChunk(iter, n); + } catch (Exception e) { + throw new CompletionException(e); + } + }, executor); + } + + /** + * Synchronous fetch of up to 'n' items. + */ + private List fetchChunk(ExceptionIterator iter, int n) throws E { + List chunk = new ArrayList<>(n); + while (chunk.size() < n && iter.hasNext()) { + chunk.add(iter.next()); + } + return chunk; + } + + @SuppressWarnings("unchecked") + private E asE(Exception ex) { + return (E) ex; + } + +} diff --git a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/iterator/utils/ParallelMerge.java b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/iterator/utils/ParallelMerge.java new file mode 100644 index 000000000..35b2a5d56 --- /dev/null +++ b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/iterator/utils/ParallelMerge.java @@ -0,0 +1,378 @@ +package com.the_qa_company.qendpoint.core.iterator.utils; + +import java.io.IOException; +import java.util.Comparator; +import java.util.List; +import java.util.NoSuchElementException; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.ForkJoinPool; +import java.util.concurrent.Future; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; +import java.util.function.Supplier; + +/** + * Abstraction for a concurrent stream of T. nextElement() => returns a Future + * that completes with the next item, or null if done. + */ +interface MergeSource { + Future nextElement() throws IOException; + + boolean hasMore() throws IOException; + + boolean exhausted(); +} + +/** + * Leaf node that pulls from a single Iterator. Each nextElement() spawns a + * task that does iterator.next(). + */ +class LeafSource implements MergeSource { + private final ExceptionIterator it; + private final ForkJoinPool pool; + private volatile boolean exhausted = false; + + LeafSource(ExceptionIterator it, ForkJoinPool pool) { + this.it = it; + this.pool = pool; + } + + @Override + public synchronized boolean hasMore() { + return !exhausted; + } + + @Override + public boolean exhausted() { + if (exhausted) + return true; + try { + if (!it.hasNext()) { + return true; + } + } catch (Exception e) { + + } + return exhausted; + } + + @Override + public Future nextElement() { + if (!hasMore()) { + return CompletableFuture.completedFuture(null); + } + CompletableFuture cf = new CompletableFuture<>(); + pool.submit(() -> { + T val = null; + synchronized (LeafSource.this) { + try { + if (!exhausted && it.hasNext()) { + val = it.next(); + } else { + exhausted = true; + } + } catch (Exception e) { + exhausted = true; + cf.completeExceptionally(e); + return; + } + } + cf.complete(val); // Will be null if exhausted + }); + return cf; + } +} + +/** + * A MergeNode that merges two children in parallel by prefetching into small + * queues. + */ +class MergeNode implements MergeSource { + private final MergeSource left; + private final MergeSource right; + private final Comparator comp; + private final ForkJoinPool pool; + + // Bounded queues to hold pre-fetched items from each child. + // In practice you might pick a different capacity or structure. + private final BlockingQueue leftQueue; + private final BlockingQueue rightQueue; + + // Flags to indicate if we've exhausted each side. + private volatile boolean leftExhausted = false; + private volatile boolean rightExhausted = false; + + // Constant to define how many items we prefetch from each child at a time. + private static final int PREFETCH_CAPACITY = 4; + + MergeNode(MergeSource left, MergeSource right, Comparator comp, ForkJoinPool pool) { + this.left = left; + this.right = right; + this.comp = comp; + this.pool = pool; + // A small queue for each side: + this.leftQueue = new LinkedBlockingQueue<>(PREFETCH_CAPACITY); + this.rightQueue = new LinkedBlockingQueue<>(PREFETCH_CAPACITY); + + // Kick off initial fill + ensurePrefetch(left, leftQueue, () -> leftExhausted); + ensurePrefetch(right, rightQueue, () -> rightExhausted); + } + + /** + * We have more if either queue is non-empty or that side can still produce + * more. + */ + @Override + public boolean hasMore() { + if (!leftQueue.isEmpty() || !rightQueue.isEmpty()) { + return true; + } + if ((leftExhausted || left.exhausted()) && (rightExhausted || right.exhausted())) { + return false; + } + return true; + } + + @Override + public boolean exhausted() { + return !hasMore(); + } + + @Override + public Future nextElement() throws IOException { + if (!hasMore()) { + return CompletableFuture.completedFuture(null); + } + + CompletableFuture cf = new CompletableFuture<>(); + + // We'll pick from the heads of both queues (blocking if empty). + // But to remain asynchronous, we do that in a pool thread: + pool.submit(() -> { + try { + // Wait for an item from each queue if available, or null if + // side is exhausted: + left.exhausted(); + T leftVal = pollOrNull(leftQueue, () -> leftExhausted || left.exhausted()); + T rightVal = pollOrNull(rightQueue, () -> rightExhausted || right.exhausted()); + + // If both sides are null => everything is exhausted + if (leftVal == null && rightVal == null) { + cf.complete(null); + return; + } + if (leftVal != null && rightVal == null) { + // only left side had an item + cf.complete(leftVal); + // Re‐prefetch next from left + ensurePrefetch(left, leftQueue, () -> leftExhausted); + return; + } + if (leftVal == null && rightVal != null) { + // only right side had an item + cf.complete(rightVal); + ensurePrefetch(right, rightQueue, () -> rightExhausted); + return; + } + // Both are non-null. Pick the smaller, put the other back in + // its queue + if (comp.compare(leftVal, rightVal) <= 0) { + // leftVal is chosen + cf.complete(leftVal); + // Put the rightVal back into the rightQueue (front) + rightQueue.put(rightVal); + // Now we can refill left side again + ensurePrefetch(left, leftQueue, () -> leftExhausted); + } else { + // rightVal is chosen + cf.complete(rightVal); + // Put the leftVal back + leftQueue.put(leftVal); + // Refill right side + ensurePrefetch(right, rightQueue, () -> rightExhausted); + } + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + cf.completeExceptionally(e); + } catch (Exception e) { + cf.completeExceptionally(e); + } + }); + + return cf; + } + + /** + * Poll one item from the queue. If the queue is empty but not exhausted, we + * block. If it's exhausted and empty, return null. + */ + private T pollOrNull(BlockingQueue queue, Supplier isExhausted) throws InterruptedException { + // If queue is non-empty, take() won't block long. + // If it's empty but not exhausted, we might wait for the next item + // (unless no more is coming). + while (true) { + if (!queue.isEmpty()) { + return queue.take(); + } + if (isExhausted.get()) { + // The child can't produce more + return null; + } + // If not exhausted and the queue is empty, + // we wait a bit to see if new items arrive from prefetch + // (Though typically ensurePrefetch will produce them soon.) + Thread.sleep(1); // simplistic small sleep; or use e.g. + // queue.poll(timeout) + } + } + + /** + * Ensures each child is prefetching new items up to the queue's capacity, + * asynchronously. + */ + private void ensurePrefetch(MergeSource child, BlockingQueue queue, Supplier exhaustedFlag) { + if (exhaustedFlag.get()) { + return; // already exhausted + } + + // While the queue still has capacity, request the next item. + // We'll do this in a loop (but asynchronously) so that we fill up to + // capacity. + pool.submit(() -> { + try { + while (!exhaustedFlag.get() && !exhausted() && queue.remainingCapacity() > 0) { + + // fetch next item + Future fut = child.nextElement(); + T val = fut.get(10, TimeUnit.SECONDS); // block in a pool + // thread + if (val == null) { + // child exhausted + setExhausted(exhaustedFlag); + break; + } + queue.put(val); + + } + } catch (Exception e) { + // Mark exhausted or propagate error somehow + setExhausted(exhaustedFlag); + } + }); + } + + private synchronized void setExhausted(Supplier exhaustedFlag) { + // Ugly but easy: if exhaustedFlag points to leftExhausted, set it, else + // set rightExhausted + // A better design might store a boolean or do a callback. + com.github.jsonldjava.shaded.com.google.common.base.Supplier isLeftExhausted = this::isLeftExhausted; + if (exhaustedFlag == isLeftExhausted) { + leftExhausted = true; + } else { + rightExhausted = true; + } + } + + private boolean isLeftExhausted() { + return leftExhausted; + } + + private boolean isRightExhausted() { + return rightExhausted; + } +} + +/** + * Build a balanced merge tree from a list of Iterators. + */ +class ParallelMergeBuilder { + public static MergeSource buildMergeTree( + List> iterators, Comparator comparator, ForkJoinPool pool) { + + int n = iterators.size(); + if (n == 0) { + return new MergeSource<>() { + @Override + public Future nextElement() { + return CompletableFuture.completedFuture(null); + } + + @Override + public boolean hasMore() { + return false; + } + + @Override + public boolean exhausted() { + return true; + } + }; + } + if (n == 1) { + return new LeafSource<>(iterators.get(0), pool); + } + // Split in half + int mid = n / 2; + MergeSource left = buildMergeTree(iterators.subList(0, mid), comparator, pool); + MergeSource right = buildMergeTree(iterators.subList(mid, n), comparator, pool); + return new MergeNode<>(left, right, comparator, pool); + } +} + +/** + * Convert a MergeSource into a normal Iterator. + */ +class ParallelMergeIterator implements ExceptionIterator { + private final MergeSource root; + private T nextItem; + + ParallelMergeIterator(MergeSource root) { + this.root = root; +// fetchNext(); + } + + private void fetchNext() throws IOException { + if (!root.hasMore()) { + nextItem = null; + return; + } + try { + nextItem = root.nextElement().get(); + } catch (InterruptedException | ExecutionException e) { + if (e.getCause() instanceof IOException) { + throw (IOException) e.getCause(); + } + throw new RuntimeException(e); + } + } + + @Override + public boolean hasNext() { + return nextItem != null; + } + + @Override + public T next() throws IOException { + if (!hasNext()) { + throw new NoSuchElementException(); + } + T ret = nextItem; + fetchNext(); + return ret; + } +} + +public class ParallelMerge { + + public static ExceptionIterator parallelMergeJoin( + List> iterators, Comparator comparator) { + + ForkJoinPool pool = ForkJoinPool.commonPool(); + MergeSource root = ParallelMergeBuilder.buildMergeTree(iterators, comparator, pool); + return new ParallelMergeIterator<>(root); + } +} From c982aaadec860cac823f79589a5e4a484b2cda1b Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ha=CC=8Avard=20Ottestad?= Date: Tue, 18 Feb 2025 20:59:31 +0100 Subject: [PATCH 18/28] fix bytestring --- .../core/util/string/ByteString.java | 79 ++----------------- 1 file changed, 7 insertions(+), 72 deletions(-) diff --git a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/util/string/ByteString.java b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/util/string/ByteString.java index 2f60f3350..e795ae8d7 100644 --- a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/util/string/ByteString.java +++ b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/util/string/ByteString.java @@ -1,11 +1,5 @@ package com.the_qa_company.qendpoint.core.util.string; -import jdk.incubator.vector.ByteVector; -import jdk.incubator.vector.ShortVector; -import jdk.incubator.vector.VectorMask; -import jdk.incubator.vector.VectorOperators; -import jdk.incubator.vector.VectorSpecies; - import java.util.Arrays; /** @@ -50,21 +44,6 @@ static ByteString copy(CharSequence csq) { */ byte[] getBuffer(); -// @Override -// default int compareTo(ByteString other) { -// int n = Math.min(length(), other.length()); -// int k = 0; -// while (k < n) { -// char c1 = charAt(k); -// char c2 = other.charAt(k); -// if (c1 != c2) { -// return c1 - c2; -// } -// k++; -// } -// return length() - other.length(); -// } - /* * (non-Javadoc) * @see java.lang.Comparable#compareTo(java.lang.Object) @@ -72,28 +51,15 @@ static ByteString copy(CharSequence csq) { @Override default int compareTo(ByteString other) { int n = Math.min(length(), other.length()); - - if (n == 0) { - return length() - other.length(); + if (n < 128) { + return naive(other, n); } + return vector(other, n); + } + private int vector(ByteString other, int n) { byte[] buffer = getBuffer(); byte[] buffer1 = other.getBuffer(); - - if (n < 64) { - return naive(other, n, buffer, buffer1); - } - - int i = charAt(0) - other.charAt(0); - if (i != 0) { - return i; - } - - return vector(other, n, buffer, buffer1); - - } - - private int vector(ByteString other, int n, byte[] buffer, byte[] buffer1) { int mismatch = Arrays.mismatch(buffer, buffer1); if (mismatch == -1 || mismatch >= n) { return length() - other.length(); @@ -101,39 +67,8 @@ private int vector(ByteString other, int n, byte[] buffer, byte[] buffer1) { return charAt(mismatch) - other.charAt(mismatch); } - private int naive(ByteString other, int n, byte[] buffer, byte[] buffer1) { - for (int i = 0; i < 32 && i < n; i++) { - if (buffer[i] != buffer1[i]) { - return charAt(i) - other.charAt(i); - } - } - return length() - other.length(); - } - - default int mismatchVectorByte(byte[] byteData1, byte[] byteData2) { - int length = Math.min(byteData1.length, byteData2.length); - int index = 0; - for (; index < ByteVector.SPECIES_PREFERRED.loopBound(length); index += ByteVector.SPECIES_PREFERRED.length()) { - ByteVector vector1 = ByteVector.fromArray(ByteVector.SPECIES_PREFERRED, byteData1, index); - ByteVector vector2 = ByteVector.fromArray(ByteVector.SPECIES_PREFERRED, byteData2, index); - VectorMask mask = vector1.compare(VectorOperators.NE, vector2); - if (mask.anyTrue()) { - return index + mask.firstTrue(); - } - } - // process the tail - int mismatch = -1; - for (int i = index; i < length; ++i) { - if (byteData1[i] != byteData2[i]) { - mismatch = i; - break; - } - } - return mismatch; - } - - private int extracted(ByteString other, int mismatch, int n) { - int k = mismatch; + private int naive(ByteString other, int n) { + int k = 0; while (k < n) { char c1 = charAt(k); char c2 = other.charAt(k); From 5f3ec78a6160548c2d1ad7fab7cc9d25508ab6cb Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ha=CC=8Avard=20Ottestad?= Date: Wed, 19 Feb 2025 12:55:15 +0100 Subject: [PATCH 19/28] wip --- .../impl/diskimport/SectionCompressor.java | 2 +- .../utils/AsyncExceptionIterator.java | 12 + .../iterator/utils/AsyncIteratorFetcher.java | 2 +- .../iterator/utils/AsyncPreFetchIterator.java | 116 +++++++ .../utils/AsyncToSyncExceptionIterator.java | 60 ++++ .../core/iterator/utils/MergeJoinZipper.java | 80 +++++ .../utils/ParallelZipperAsyncIterator.java | 114 ++++++ .../iterator/utils/PipedCopyIterator.java | 2 +- .../iterator/utils/ZipperAsyncIterator.java | 82 +++++ .../compress/CompressNodeMergeIterator.java | 4 + .../compress/CompressTripleMergeIterator.java | 3 + .../core/util/io/compress/CompressUtil.java | 10 +- .../listener/MultiThreadListenerConsole.java | 2 +- .../core/util/string/ByteString.java | 103 +++++- .../core/util/string/ByteStringUtil.java | 78 +++-- .../core/util/string/ReplazableString.java | 56 +-- .../core/iterator/utils/MergeJoinTests.java | 326 ++++++++++++++++++ 17 files changed, 986 insertions(+), 66 deletions(-) create mode 100644 qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/iterator/utils/AsyncExceptionIterator.java create mode 100644 qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/iterator/utils/AsyncPreFetchIterator.java create mode 100644 qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/iterator/utils/AsyncToSyncExceptionIterator.java create mode 100644 qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/iterator/utils/MergeJoinZipper.java create mode 100644 qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/iterator/utils/ParallelZipperAsyncIterator.java create mode 100644 qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/iterator/utils/ZipperAsyncIterator.java create mode 100644 qendpoint-core/src/test/java/com/the_qa_company/qendpoint/core/iterator/utils/MergeJoinTests.java diff --git a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/hdt/impl/diskimport/SectionCompressor.java b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/hdt/impl/diskimport/SectionCompressor.java index a183821b8..09e208473 100644 --- a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/hdt/impl/diskimport/SectionCompressor.java +++ b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/hdt/impl/diskimport/SectionCompressor.java @@ -60,7 +60,7 @@ public SectionCompressor(CloseSuppressPath baseFileName, AsyncIteratorFetcher { + CompletableFuture nextFuture(); +} diff --git a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/iterator/utils/AsyncIteratorFetcher.java b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/iterator/utils/AsyncIteratorFetcher.java index d45362264..68f191968 100644 --- a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/iterator/utils/AsyncIteratorFetcher.java +++ b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/iterator/utils/AsyncIteratorFetcher.java @@ -18,7 +18,7 @@ public class AsyncIteratorFetcher implements Supplier { private final Lock lock = new ReentrantLock(); private boolean end; - volatile ConcurrentLinkedQueue queue = new ConcurrentLinkedQueue<>(); + ConcurrentLinkedQueue queue = new ConcurrentLinkedQueue<>(); public AsyncIteratorFetcher(Iterator iterator) { this.iterator = iterator; diff --git a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/iterator/utils/AsyncPreFetchIterator.java b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/iterator/utils/AsyncPreFetchIterator.java new file mode 100644 index 000000000..f435123c0 --- /dev/null +++ b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/iterator/utils/AsyncPreFetchIterator.java @@ -0,0 +1,116 @@ +package com.the_qa_company.qendpoint.core.iterator.utils; + +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CompletionException; +import java.util.concurrent.Executor; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicReference; + +/** + * A wrapper that pre-fetches from a synchronous ExceptionIterator + * asynchronously. - Creates or uses a provided Executor (potentially a + * VirtualThreadPerTaskExecutor). - Encourages calling cancel() or close() if + * partial consumption is likely. + */ +class AsyncPreFetchIterator implements AsyncExceptionIterator, AutoCloseable { + + private final ExceptionIterator iterator; + + // Holds the current fetch (chained) so it happens strictly in sequence. + private CompletableFuture currentFetch; + + private final AtomicBoolean hasMore = new AtomicBoolean(true); + private final AtomicBoolean cancelled = new AtomicBoolean(false); + private final AtomicReference exception = new AtomicReference<>(null); + + private final Executor executor; + private final boolean ownExecutor; + + public AsyncPreFetchIterator(ExceptionIterator iterator) { + this(iterator, null); + } + + public AsyncPreFetchIterator(ExceptionIterator iterator, Executor executor) { + this.iterator = iterator; + if (executor == null) { + this.executor = Executors.newVirtualThreadPerTaskExecutor(); + this.ownExecutor = true; + } else { + this.executor = executor; + this.ownExecutor = false; + } + // Schedule the initial fetch once + this.currentFetch = fetchNext(null); + } + + /** + * nextFuture() returns the current future, then updates currentFetch so + * that the next fetch is chained after the current future completes. + */ + @Override + public synchronized CompletableFuture nextFuture() { + CompletableFuture result = currentFetch; + + // Chain the "next" fetch to happen strictly after the current result is + // done: + currentFetch = result.thenCompose(ignored -> fetchNext(null)); + return result; + } + + /** + * A peek method if needed to see the current element without advancing. + * This is safe: we do not schedule an additional fetch unless nextFuture() + * is called. + */ + public synchronized CompletableFuture peekFuture() { + return currentFetch; + } + + /** + * fetchNext() returns a future that (when run) checks hasNext() & next(). + * It's always chained after the previous fetch, preventing parallel calls. + */ + private CompletableFuture fetchNext(T ignored) { + // If already cancelled or exception set, do not schedule more tasks. + if (cancelled.getAcquire() || exception.getAcquire() != null) { + return CompletableFuture.completedFuture(null); + } + return CompletableFuture.supplyAsync(() -> { + try { + if (iterator.hasNext()) { + return iterator.next(); + } else { + hasMore.setRelease(false); + return null; + } + } catch (Exception ex) { + exception.compareAndSet(null, ex); + cancelled.setRelease(true); + throw new CompletionException(ex); + } + }, executor); + } + + /** + * Cancel and prevent further scheduling. + */ + public void cancel() { + cancelled.setRelease(true); + if (currentFetch != null) { + currentFetch.cancel(true); + } + } + + /** + * Closes resources. If we own the executor, we shut it down. + */ + @Override + public void close() { + cancel(); + if (ownExecutor && executor instanceof ExecutorService) { + ((ExecutorService) executor).shutdownNow(); + } + } +} diff --git a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/iterator/utils/AsyncToSyncExceptionIterator.java b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/iterator/utils/AsyncToSyncExceptionIterator.java new file mode 100644 index 000000000..5bf0f9d30 --- /dev/null +++ b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/iterator/utils/AsyncToSyncExceptionIterator.java @@ -0,0 +1,60 @@ +package com.the_qa_company.qendpoint.core.iterator.utils; + +import java.util.NoSuchElementException; +import java.util.concurrent.CompletionException; + +public class AsyncToSyncExceptionIterator implements ExceptionIterator { + + private final AsyncExceptionIterator asyncIterator; + private T nextValue; + private boolean hasPrefetched = false; + private boolean finished = false; + + public AsyncToSyncExceptionIterator(AsyncExceptionIterator asyncIterator) { + this.asyncIterator = asyncIterator; + } + + @Override + public boolean hasNext() throws E { + if (finished) { + return false; + } + // If we haven't prefetched yet, do so + if (!hasPrefetched) { + fetchNext(); + hasPrefetched = true; + } + return !finished; + } + + @Override + public T next() throws E { + if (!hasNext()) { + throw new NoSuchElementException("Iterator exhausted"); + } + // Return the prefetched value + T valueToReturn = nextValue; + // Immediately fetch the next one + fetchNext(); + return valueToReturn; + } + + private void fetchNext() throws E { + try { + T result = asyncIterator.nextFuture().join(); + if (result == null) { + finished = true; + nextValue = null; + } else { + nextValue = result; + } + } catch (CompletionException ce) { + Throwable cause = ce.getCause(); + if (cause instanceof Exception) { + // noinspection unchecked + throw (E) cause; + } + throw ce; + } + } +} diff --git a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/iterator/utils/MergeJoinZipper.java b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/iterator/utils/MergeJoinZipper.java new file mode 100644 index 000000000..46d1d5332 --- /dev/null +++ b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/iterator/utils/MergeJoinZipper.java @@ -0,0 +1,80 @@ +package com.the_qa_company.qendpoint.core.iterator.utils; + +import java.util.ArrayList; +import java.util.Comparator; +import java.util.List; + +/** + * MergeJoinZipper builds a balanced merge tree from a list of sorted + * ExceptionIterator instances. The resulting ExceptionIterator is synchronous, + * but internally uses asynchronous prefetching and merging. + */ +public class MergeJoinZipper { + + /** + * @param iterators A list of sorted, synchronous ExceptionIterator + * objects. + * @param comparator The comparator used to merge them. + * @return A final ExceptionIterator that merges all input in sorted order. + */ + public static ExceptionIterator buildMergeTree( + List> iterators, Comparator comparator) { + if (iterators.isEmpty()) { + return new EmptyExceptionIterator<>(); + } + if (iterators.size() == 1) { + return wrapAsync(iterators.get(0)); + } + + // 1) Wrap each synchronous iterator in an AsyncPreFetchIterator, + // but store them in a list as AsyncExceptionIterator. + List> asyncIters = new ArrayList<>(); + for (ExceptionIterator it : iterators) { + asyncIters.add(new AsyncPreFetchIterator<>(it)); + } + + // 2) Pairwise merge them until only one remains + while (asyncIters.size() > 1) { + List> merged = new ArrayList<>(); + for (int i = 0; i < asyncIters.size(); i += 2) { + if (i + 1 < asyncIters.size()) { + AsyncExceptionIterator left = asyncIters.get(i); + AsyncExceptionIterator right = asyncIters.get(i + 1); + // Now you can merge them in a ZipperAsyncIterator + merged.add(new ParallelZipperAsyncIterator<>(left, right, comparator)); + } else { + merged.add(asyncIters.get(i)); + } + } + asyncIters = merged; + } + + // 3) Wrap the final AsyncExceptionIterator in a synchronous + // AsyncToSyncExceptionIterator + return new AsyncToSyncExceptionIterator<>(asyncIters.get(0)); + } + + /** + * Helper method for the single-iterator case. + */ + private static ExceptionIterator wrapAsync(ExceptionIterator iterator) { + AsyncPreFetchIterator async = new AsyncPreFetchIterator<>(iterator); + return new AsyncToSyncExceptionIterator<>(async); + } +} + +/** + * A simple empty iterator implementation (used if the list of iterators is + * empty). + */ +class EmptyExceptionIterator implements ExceptionIterator { + @Override + public boolean hasNext() { + return false; + } + + @Override + public T next() { + throw new java.util.NoSuchElementException("Empty iterator"); + } +} diff --git a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/iterator/utils/ParallelZipperAsyncIterator.java b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/iterator/utils/ParallelZipperAsyncIterator.java new file mode 100644 index 000000000..c91c6e13e --- /dev/null +++ b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/iterator/utils/ParallelZipperAsyncIterator.java @@ -0,0 +1,114 @@ +package com.the_qa_company.qendpoint.core.iterator.utils; + +import java.util.Comparator; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CompletionException; + +/** + * ParallelZipperAsyncIterator merges two sorted AsyncExceptionIterator sources + * into a single sorted stream. Both sides always have exactly one fetch in + * progress, which maximizes concurrency. The main steps in nextFuture(): 1. + * Wait for leftCurrent and rightCurrent to complete, giving (leftVal, + * rightVal). 2. Create a Result object indicating which value to return and + * which side(s) to advance. 3. Then compose (flatten) that into a single future + * that schedules new fetches for whichever side was consumed, and returns the + * chosen value to the caller. + */ +public class ParallelZipperAsyncIterator implements AsyncExceptionIterator { + + private final AsyncExceptionIterator left; // The left input stream + private final AsyncExceptionIterator right; // The right input stream + private final Comparator comparator; // Comparator for sorting + + // Each side holds one "current" item in flight + private CompletableFuture leftCurrent; + private CompletableFuture rightCurrent; + + /** + * Constructs a ParallelZipperAsyncIterator from two AsyncExceptionIterator + * sources. Immediately fetches one element from each side. + */ + public ParallelZipperAsyncIterator(AsyncExceptionIterator left, AsyncExceptionIterator right, + Comparator comparator) { + this.left = left; + this.right = right; + this.comparator = comparator; + + // Start fetching one item on each side + this.leftCurrent = left.nextFuture(); + this.rightCurrent = right.nextFuture(); + } + + /** + * nextFuture(): 1. Waits for both sides' current futures to complete -> + * (leftVal, rightVal). 2. Decides which item to return and which side to + * advance, building a small Result object. 3. thenCompose on that Result to + * schedule side fetches if needed and return the chosen value. + */ + @Override + public CompletableFuture nextFuture() { + // Combine the two futures to get leftVal and rightVal once both + // complete + CompletableFuture combined = leftCurrent.thenCombine(rightCurrent, (leftVal, rightVal) -> { + if (leftVal == null && rightVal == null) { + // Both sides exhausted + return new Result(null, false, false); + } else if (leftVal == null) { + // Left exhausted, return rightVal, advance right + return new Result(rightVal, false, true); + } else if (rightVal == null) { + // Right exhausted, return leftVal, advance left + return new Result(leftVal, true, false); + } else { + // Both non-null, pick the smaller + if (comparator.compare(leftVal, rightVal) <= 0) { + // Use left + return new Result(leftVal, true, false); + } else { + // Use right + return new Result(rightVal, false, true); + } + } + }); + + // Now we flatten combined (a Future) into a Future. + // In the .thenCompose, we schedule new fetches for whichever side was + // consumed + // and return the chosen value (which might be null if both exhausted). + return combined.thenCompose(res -> { + // If res.value == null => both are exhausted + if (res.value == null) { + return CompletableFuture.completedFuture(null); + } else { + // If we used left side => schedule a new fetch for left + if (res.advanceLeft) { + leftCurrent = left.nextFuture(); + } + // If we used right side => schedule a new fetch for right + if (res.advanceRight) { + rightCurrent = right.nextFuture(); + } + return CompletableFuture.completedFuture(res.value); + } + }).exceptionally(ex -> { + // If an exception occurs, rethrow it as a CompletionException + throw new CompletionException(ex); + }); + } + + /** + * A small helper class that indicates which item we decided to return, and + * whether we want to advance the left or right side. + */ + private class Result { + final T value; + final boolean advanceLeft; + final boolean advanceRight; + + Result(T value, boolean advanceLeft, boolean advanceRight) { + this.value = value; + this.advanceLeft = advanceLeft; + this.advanceRight = advanceRight; + } + } +} diff --git a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/iterator/utils/PipedCopyIterator.java b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/iterator/utils/PipedCopyIterator.java index ab9aa05a0..da510fd15 100644 --- a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/iterator/utils/PipedCopyIterator.java +++ b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/iterator/utils/PipedCopyIterator.java @@ -110,7 +110,7 @@ public T get() { } } - private final ArrayBlockingQueue> queue = new ArrayBlockingQueue<>(16); + private final ArrayBlockingQueue> queue = new ArrayBlockingQueue<>(128); private T next; private boolean end; diff --git a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/iterator/utils/ZipperAsyncIterator.java b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/iterator/utils/ZipperAsyncIterator.java new file mode 100644 index 000000000..920825956 --- /dev/null +++ b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/iterator/utils/ZipperAsyncIterator.java @@ -0,0 +1,82 @@ +package com.the_qa_company.qendpoint.core.iterator.utils; + +import java.util.Comparator; +import java.util.concurrent.CompletableFuture; + +/** + * ZipperAsyncIterator merges two sorted AsyncExceptionIterator streams into a + * single sorted stream. It stores a "buffer" future for each side, retrieves + * them in parallel, and compares their values. + */ +public class ZipperAsyncIterator implements AsyncExceptionIterator { + private final AsyncExceptionIterator left; + private final AsyncExceptionIterator right; + private final Comparator comparator; + + // Buffers for each side. Each call to nextFuture() will compare these + // values and advance the used side. + private CompletableFuture leftBuffer; + private CompletableFuture rightBuffer; + + /** + * Constructs a ZipperAsyncIterator from two AsyncExceptionIterators and a + * comparator. We initialize each side's buffer by calling nextFuture() + * once. + */ + public ZipperAsyncIterator(AsyncExceptionIterator left, AsyncExceptionIterator right, + Comparator comparator) { + this.left = left; + this.right = right; + this.comparator = comparator; + + // Initialize each buffer with one fetched value. + this.leftBuffer = left.nextFuture(); + this.rightBuffer = right.nextFuture(); + } + + /** + * nextFuture() returns a future that, when complete, yields the next merged + * element (or null if both sides are exhausted). We compare the two + * buffered values and advance only the side whose element is chosen. + */ + @Override + public CompletableFuture nextFuture() { + // Combine the two buffer futures into a single future-of-a-future. + // When both buffers resolve, compare them to see which side's value to + // consume. + CompletableFuture> combined = leftBuffer.thenCombine(rightBuffer, (leftVal, rightVal) -> { + if (leftVal == null && rightVal == null) { + // Both sides are exhausted + return CompletableFuture.completedFuture(null); + } else if (leftVal == null) { + // Left is exhausted, so return rightVal and advance right side + CompletableFuture toReturn = CompletableFuture.completedFuture(rightVal); + rightBuffer = right.nextFuture(); // fetch the next from the + // right + return toReturn; + } else if (rightVal == null) { + // Right is exhausted, so return leftVal and advance left side + CompletableFuture toReturn = CompletableFuture.completedFuture(leftVal); + leftBuffer = left.nextFuture(); // fetch the next from the left + return toReturn; + } else { + // Both sides have a value; compare them + if (comparator.compare(leftVal, rightVal) <= 0) { + // left is smaller (or equal) + CompletableFuture toReturn = CompletableFuture.completedFuture(leftVal); + leftBuffer = left.nextFuture(); // refill from left side + return toReturn; + } else { + // right is smaller + CompletableFuture toReturn = CompletableFuture.completedFuture(rightVal); + rightBuffer = right.nextFuture(); // refill from right side + return toReturn; + } + } + }); + + // combined is a Future>. We flatten it to Future with + // thenCompose. + return combined.thenCompose(f -> f); + } +} diff --git a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/util/io/compress/CompressNodeMergeIterator.java b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/util/io/compress/CompressNodeMergeIterator.java index b8c653377..206b275e4 100644 --- a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/util/io/compress/CompressNodeMergeIterator.java +++ b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/util/io/compress/CompressNodeMergeIterator.java @@ -1,10 +1,13 @@ package com.the_qa_company.qendpoint.core.util.io.compress; +import com.the_qa_company.qendpoint.core.iterator.utils.MergeJoinZipper; import com.the_qa_company.qendpoint.core.triples.IndexedNode; import com.the_qa_company.qendpoint.core.iterator.utils.ExceptionIterator; import com.the_qa_company.qendpoint.core.iterator.utils.MergeExceptionIterator; +import com.the_qa_company.qendpoint.core.triples.TripleIDComparator; import java.io.IOException; +import java.util.Arrays; /** * Version of {@link MergeExceptionIterator} with {@link IndexedNode} @@ -20,6 +23,7 @@ public CompressNodeMergeIterator(ExceptionIterator in1 public static > ExceptionIterator buildOfTree( T[] lst) { +// return MergeJoinZipper.buildMergeTree(Arrays.asList(lst), IndexedNode::compareTo); return buildOfTree(it -> it, IndexedNode::compareTo, lst, 0, lst.length); } } diff --git a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/util/io/compress/CompressTripleMergeIterator.java b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/util/io/compress/CompressTripleMergeIterator.java index a8ab0ee19..dbefcebfd 100644 --- a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/util/io/compress/CompressTripleMergeIterator.java +++ b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/util/io/compress/CompressTripleMergeIterator.java @@ -1,12 +1,14 @@ package com.the_qa_company.qendpoint.core.util.io.compress; import com.the_qa_company.qendpoint.core.enums.TripleComponentOrder; +import com.the_qa_company.qendpoint.core.iterator.utils.MergeJoinZipper; import com.the_qa_company.qendpoint.core.triples.TripleID; import com.the_qa_company.qendpoint.core.triples.TripleIDComparator; import com.the_qa_company.qendpoint.core.iterator.utils.ExceptionIterator; import com.the_qa_company.qendpoint.core.iterator.utils.MergeExceptionIterator; import java.io.IOException; +import java.util.Arrays; /** * Version of {@link MergeExceptionIterator} with {@link TripleID} @@ -22,6 +24,7 @@ public CompressTripleMergeIterator(ExceptionIterator in1, public static > ExceptionIterator buildOfTree( T[] lst, TripleComponentOrder order) { +// return MergeJoinZipper.buildMergeTree(Arrays.asList(lst), TripleIDComparator.getComparator(order)); return buildOfTree(it -> it, TripleIDComparator.getComparator(order), lst, 0, lst.length); } } diff --git a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/util/io/compress/CompressUtil.java b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/util/io/compress/CompressUtil.java index 4c4c8a4e9..a548a938f 100644 --- a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/util/io/compress/CompressUtil.java +++ b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/util/io/compress/CompressUtil.java @@ -193,9 +193,13 @@ public boolean hasNext() { while (it.hasNext()) { IndexedNode node = it.next(); ByteString next = node.getNode(); - int cmp = prev.compareTo(next); - assert cmp <= 0 : "bad order : " + prev + " > " + next; - if (cmp == 0) { + boolean equals = prev.equals(next); + + assert prev.compareTo(next) <= 0 : "bad order : " + prev + " > " + next; + assert !equals || prev.compareTo(next) == 0 : "bad equals : " + prev + " != " + next; + assert equals || prev.compareTo(next) != 0 : "bad compare : " + prev + " != " + next; + + if (equals) { if (!prev.isEmpty() || prevRead) { // same as previous, ignore assert this.id != node.getIndex() : "same index and prevIndex"; diff --git a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/util/listener/MultiThreadListenerConsole.java b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/util/listener/MultiThreadListenerConsole.java index 0e5ba1023..a14c51b64 100644 --- a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/util/listener/MultiThreadListenerConsole.java +++ b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/util/listener/MultiThreadListenerConsole.java @@ -167,7 +167,7 @@ public void removeLast() { Thread thread = new Thread(() -> { while (true) { try { - Thread.sleep(500); + Thread.sleep(2000); render(); } catch (InterruptedException e) { break; diff --git a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/util/string/ByteString.java b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/util/string/ByteString.java index e795ae8d7..44a794881 100644 --- a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/util/string/ByteString.java +++ b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/util/string/ByteString.java @@ -1,6 +1,7 @@ package com.the_qa_company.qendpoint.core.util.string; import java.util.Arrays; +import java.util.concurrent.atomic.LongAdder; /** * ByteString char sequence, can't be compared with string, faster than string @@ -51,9 +52,78 @@ static ByteString copy(CharSequence csq) { @Override default int compareTo(ByteString other) { int n = Math.min(length(), other.length()); - if (n < 128) { - return naive(other, n); + switch (n) { + case 0: + return length() - other.length(); + case 1: { + char c1 = charAt(0); + char c2 = other.charAt(0); + int ret = c1 - c2; + if (ret != 0) { + return ret; + } + return length() - other.length(); } + case 2: { + char c1 = charAt(0); + char c2 = other.charAt(0); + if (c1 != c2) { + return c1 - c2; + } + c1 = charAt(1); + c2 = other.charAt(1); + if (c1 != c2) { + return c1 - c2; + } + return length() - other.length(); + } + case 3: + return naive3(other); + default: + return fastCompare(other, n); + } + + } + +// LongAdder compareLessThan8 = new LongAdder(); +// LongAdder compareLessThan4 = new LongAdder(); +// LongAdder compareVector = new LongAdder(); + + private int fastCompare(ByteString other, int n) { + +// if ((compareVector.sum() + compareLessThan8.sum()) % 1000000 == 0) { +// System.out.println("compareLessThan4: " + compareLessThan4.sum()); +// System.out.println("compareLessThan8: " + compareLessThan8.sum()); +// System.out.println("compareVector: " + compareVector.sum()); +// } + +// if (n > 20) { +// char c = charAt(0); +// if (c == 'h') { +// if (charAt(0) != other.charAt(0)) { +// return charAt(0) - other.charAt(0); +// } +// if (charAt(n / 2) != other.charAt(n / 2)) { +// n = n / 2; +// } +// } +// } + +// if (Temp.fast) { + for (int i = 0; i < 4 && i < n; i++) { + char c1 = charAt(i); + char c2 = other.charAt(i); + if (c1 != c2) { +// compareLessThan4.increment(); + return c1 - c2; + } + } + +// Temp.fast = false; +// } + +// compareVector.increment(); + return vector(other, n); } @@ -64,6 +134,9 @@ private int vector(ByteString other, int n) { if (mismatch == -1 || mismatch >= n) { return length() - other.length(); } +// if (mismatch < 8) { +// Temp.fast = true; +// } return charAt(mismatch) - other.charAt(mismatch); } @@ -80,6 +153,28 @@ private int naive(ByteString other, int n) { return length() - other.length(); } + private int naive3(ByteString other) { + char c1 = charAt(0); + char c2 = other.charAt(0); + if (c1 != c2) { + return c1 - c2; + } + + c1 = charAt(1); + c2 = other.charAt(1); + if (c1 != c2) { + return c1 - c2; + } + + c1 = charAt(2); + c2 = other.charAt(2); + if (c1 != c2) { + return c1 - c2; + } + + return length() - other.length(); + } + @Override ByteString subSequence(int start, int end); @@ -179,3 +274,7 @@ default boolean startsWith(ByteString prefix, int start) { } } + +class Temp { + volatile static boolean fast = false; +} diff --git a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/util/string/ByteStringUtil.java b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/util/string/ByteStringUtil.java index 68e2dfb4a..1fab95dbf 100644 --- a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/util/string/ByteStringUtil.java +++ b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/util/string/ByteStringUtil.java @@ -198,41 +198,54 @@ public static int longestCommonPrefix(CharSequence str1, CharSequence str2, int if (str1 instanceof ByteString && str2 instanceof ByteString) { - if (len - from < 64) { + if (len - from < 4) { + if (from == 0) { + return switch (len) { + case 1 -> str1.charAt(0) == str2.charAt(0) ? 1 : 0; + case 2 -> naiveLength2(str1, str2); + case 3 -> naiveLength3(str1, str2); + default -> naive(str1, str2, from, len); + }; + } return naive(str1, str2, from, len); } byte[] buffer = ((ByteString) str1).getBuffer(); byte[] buffer2 = ((ByteString) str2).getBuffer(); - // System.out.println("mismatch: " + i); -// if (from == 0) { -// return vector(buffer, buffer2, len); -// } else { return mismatch(from, buffer, len, buffer2); -// } - -// int delta = from; -// while (delta < len && str1.charAt(delta) == str2.charAt(delta)) { -// delta++; -// } -// // System.out.println("i: " + i); -// int i = delta - from; -// -// int i1 = missmatch-from; -// -// if (i != i1) { -//// longestCommonPrefix(str1, str2, from); -// throw new AssertionError("Mismatch: " + i + " " + i1); -// } -// -// return i1; - -// return missmatch - from; + } return naive(str1, str2, from, len); } + private static int naiveLength3(CharSequence str1, CharSequence str2) { + char c = str1.charAt(0); + char c1 = str2.charAt(0); + if (c != c1) { + return 0; + } + c = str1.charAt(1); + c1 = str2.charAt(1); + if (c != c1) { + return 1; + } + c = str1.charAt(2); + c1 = str2.charAt(2); + return c == c1 ? 3 : 2; + } + + private static int naiveLength2(CharSequence str1, CharSequence str2) { + char c = str1.charAt(0); + char c1 = str2.charAt(0); + if (c != c1) { + return 0; + } + c = str1.charAt(1); + c1 = str2.charAt(1); + return c == c1 ? 2 : 1; + } + private static int mismatch(int from, byte[] buffer, int len, byte[] buffer2) { int missmatch = Arrays.mismatch(buffer, from, len, buffer2, from, len); if (missmatch == -1) { @@ -242,15 +255,6 @@ private static int mismatch(int from, byte[] buffer, int len, byte[] buffer2) { } } - private static int vector(byte[] buffer, byte[] buffer2, int len) { - int mismatch = mismatchVectorByte(buffer, buffer2); - if (mismatch == -1 || mismatch >= len) { - return len; - } else { - return mismatch; - } - } - private static int naive(CharSequence str1, CharSequence str2, int from, int len) { int delta = from; while (delta < len && str1.charAt(delta) == str2.charAt(delta)) { @@ -259,6 +263,14 @@ private static int naive(CharSequence str1, CharSequence str2, int from, int len return delta - from; } + private static int naive(CharSequence str1, CharSequence str2) { + int delta = 0; + while (delta < 3 && str1.charAt(delta) == str2.charAt(delta)) { + delta++; + } + return delta; + } + private static int mismatchVectorByte(byte[] byteData1, byte[] byteData2) { int length = Math.min(byteData1.length, byteData2.length); int index = 0; diff --git a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/util/string/ReplazableString.java b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/util/string/ReplazableString.java index 1de26b1b3..a2d7a9aba 100644 --- a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/util/string/ReplazableString.java +++ b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/util/string/ReplazableString.java @@ -291,36 +291,44 @@ public boolean equals(Object o) { return true; } if (o instanceof CompactString) { - CompactString cmp = (CompactString) o; - if (buffer.length != cmp.data.length) { - return false; - } - - // Byte by byte comparison - int i = buffer.length; - while (i-- != 0) { - if (buffer[i] != cmp.data[i]) { - return false; - } - } - return true; + return equalsCompactString((CompactString) o); } else if (o instanceof ReplazableString cmp) { - if (this.used != cmp.used) { + return equalsReplazableString(cmp); + } else if (o instanceof CharSequence other) { + return length() == other.length() && CharSequenceComparator.getInstance().compare(this, other) == 0; + } + throw new NotImplementedException(); + } + + private boolean equalsReplazableString(ReplazableString cmp) { + if (this.used != cmp.used) { + return false; + } + + // Byte by byte comparison + int i = this.used; + while (i-- != 0) { + if (buffer[i] != cmp.buffer[i]) { return false; } + } + return true; + } - // Byte by byte comparison - int i = this.used; - while (i-- != 0) { - if (buffer[i] != cmp.buffer[i]) { - return false; - } + private boolean equalsCompactString(CompactString o) { + CompactString cmp = o; + if (used != cmp.data.length) { + return false; + } + + // Byte by byte comparison + int i = used; + while (i-- != 0) { + if (buffer[i] != cmp.data[i]) { + return false; } - return true; - } else if (o instanceof CharSequence other) { - return length() == other.length() && CharSequenceComparator.getInstance().compare(this, other) == 0; } - throw new NotImplementedException(); + return true; } /* diff --git a/qendpoint-core/src/test/java/com/the_qa_company/qendpoint/core/iterator/utils/MergeJoinTests.java b/qendpoint-core/src/test/java/com/the_qa_company/qendpoint/core/iterator/utils/MergeJoinTests.java new file mode 100644 index 000000000..2ae3d10c4 --- /dev/null +++ b/qendpoint-core/src/test/java/com/the_qa_company/qendpoint/core/iterator/utils/MergeJoinTests.java @@ -0,0 +1,326 @@ +package com.the_qa_company.qendpoint.core.iterator.utils; + +import org.junit.Test; +import org.junit.Before; +import org.junit.After; +import static org.junit.Assert.*; + +import java.util.*; +import java.util.concurrent.*; +import java.util.stream.Collectors; + +/** + * Tests for the MergeJoinZipper and related asynchronous merge join iterators, + * using JUnit 4. + */ +public class MergeJoinTests { + + // --- Helper iterator classes for testing --- + + /** + * A simple test iterator that wraps a List and implements + * ExceptionIterator. + */ + public static class TestListIterator implements ExceptionIterator { + private final List list; + private int index = 0; + + public TestListIterator(List list) { + this.list = list; + } + + @Override + public boolean hasNext() { + return index < list.size(); + } + + @Override + public T next() { + if (!hasNext()) { + throw new NoSuchElementException("Iterator exhausted"); + } + return list.get(index++); + } + } + + /** + * A helper iterator that throws an exception at a specified index. + */ + public static class FailingTestListIterator implements ExceptionIterator { + private final List list; + private final int failAt; + private int index = 0; + + public FailingTestListIterator(List list, int failAt) { + this.list = list; + this.failAt = failAt; + } + + @Override + public boolean hasNext() { + return index < list.size(); + } + + @Override + public T next() { + if (index == failAt) { + throw new RuntimeException("Intentional failure at index " + failAt); + } + if (!hasNext()) { + throw new NoSuchElementException("Iterator exhausted"); + } + return list.get(index++); + } + } + + /** + * Helper method: consumes an ExceptionIterator and returns its elements as + * a List. + */ + private List consumeIterator(ExceptionIterator iterator) throws Exception { + List result = new ArrayList<>(); + while (iterator.hasNext()) { + result.add(iterator.next()); + } + return result; + } + + /** + * Helper to wrap a synchronous ExceptionIterator in an + * AsyncPreFetchIterator and then in an AsyncToSyncExceptionIterator. + */ + private ExceptionIterator wrapSync(ExceptionIterator iter) { + AsyncPreFetchIterator async = new AsyncPreFetchIterator<>(iter); + return new AsyncToSyncExceptionIterator<>(async); + } + + // --- Tests using MergeJoinZipper --- + + @Test + public void testMergeJoinZipper_SimpleTwoIterators() throws Exception { + List list1 = Arrays.asList(1, 3, 5); + List list2 = Arrays.asList(2, 4, 6); + + List> input = new ArrayList<>(); + input.add(new TestListIterator<>(list1)); + input.add(new TestListIterator<>(list2)); + + ExceptionIterator merged = MergeJoinZipper.buildMergeTree(input, Integer::compareTo); + + List actual = consumeIterator(merged); + List expected = Arrays.asList(1, 2, 3, 4, 5, 6); + assertEquals(expected, actual); + } + + @Test + public void testMergeJoinZipper_OneEmptyOneNonEmpty() throws Exception { + List nonEmpty = Arrays.asList(10, 20, 30); + + List> input = new ArrayList<>(); + input.add(new TestListIterator<>(Collections.emptyList())); + input.add(new TestListIterator<>(nonEmpty)); + + ExceptionIterator merged = MergeJoinZipper.buildMergeTree(input, Integer::compareTo); + + List actual = consumeIterator(merged); + assertEquals(nonEmpty, actual); + } + + @Test + public void testMergeJoinZipper_MultipleIterators() throws Exception { + List list1 = Arrays.asList(1, 4, 8); + List list2 = Arrays.asList(2, 3, 9); + List list3 = Arrays.asList(5, 6, 7); + + List> input = Arrays.asList(new TestListIterator<>(list1), + new TestListIterator<>(list2), new TestListIterator<>(list3)); + + ExceptionIterator merged = MergeJoinZipper.buildMergeTree(input, Integer::compareTo); + + List actual = consumeIterator(merged); + List expected = Arrays.asList(1, 2, 3, 4, 5, 6, 7, 8, 9); + assertEquals(expected, actual); + } + + @Test + public void testMergeJoinZipper_EmptyList() { + ExceptionIterator merged = MergeJoinZipper.buildMergeTree(Collections.emptyList(), + Integer::compareTo); + try { + assertFalse(merged.hasNext()); + } catch (Exception e) { + fail("hasNext() should not throw exception on empty iterator."); + } + try { + merged.next(); + fail("Expected NoSuchElementException for empty iterator."); + } catch (NoSuchElementException e) { + // expected + } catch (Exception e) { + fail("Expected NoSuchElementException, but got: " + e); + } + } + + @Test + public void testMergeJoinZipper_SingleIterator() throws Exception { + List data = Arrays.asList(100, 200, 300); + List> input = Collections.singletonList(new TestListIterator<>(data)); + + ExceptionIterator merged = MergeJoinZipper.buildMergeTree(input, Integer::compareTo); + + List actual = consumeIterator(merged); + assertEquals(data, actual); + } + + @Test + public void testMergeJoinZipper_ExceptionPropagation() throws Exception { + List okList = Arrays.asList(1, 2, 3); + FailingTestListIterator failingIter = new FailingTestListIterator<>(Arrays.asList(10, 11, 12), 1); + + List> input = new ArrayList<>(); + input.add(new TestListIterator<>(okList)); + input.add(failingIter); + + ExceptionIterator merged = MergeJoinZipper.buildMergeTree(input, Integer::compareTo); + + assertEquals(1, (int) merged.next()); + assertEquals(2, (int) merged.next()); + assertEquals(3, (int) merged.next()); + try { + merged.next(); + fail("Expected exception from failing iterator."); + } catch (Exception ex) { + assertTrue(ex.getMessage().contains("Intentional failure")); + } + } + + @Test(timeout = 5000) + public void testMergeJoinZipper_Cancellation() { + ExceptionIterator slowIter = new ExceptionIterator() { + private int count = 0; + + @Override + public boolean hasNext() { + return count < 5; + } + + @Override + public Integer next() throws Exception { + Thread.sleep(200); + return count++; + } + }; + + AsyncPreFetchIterator asyncIterator = new AsyncPreFetchIterator<>(slowIter); + asyncIterator.cancel(); + CompletableFuture future = asyncIterator.nextFuture(); + try { + Integer value = future.join(); + assertNull(value); + } catch (CompletionException ce) { + assertTrue(ce.getCause() instanceof CancellationException); + } finally { + asyncIterator.close(); + } + } + + @Test + public void testMergeJoinZipper_RepeatedHasNext() throws Exception { + List data = Arrays.asList(5, 10, 15); + ExceptionIterator iter = wrapSync(new TestListIterator<>(data)); + + assertTrue(iter.hasNext()); + assertTrue(iter.hasNext()); + assertEquals(5, (int) iter.next()); + + assertTrue(iter.hasNext()); + assertTrue(iter.hasNext()); + assertEquals(10, (int) iter.next()); + + assertTrue(iter.hasNext()); + assertEquals(15, (int) iter.next()); + assertFalse(iter.hasNext()); + } + + @Test + public void testMergeJoinZipper_NextAfterExhaustion() throws Exception { + List data = Arrays.asList(100); + ExceptionIterator iter = wrapSync(new TestListIterator<>(data)); + + assertTrue(iter.hasNext()); + assertEquals(100, (int) iter.next()); + assertFalse(iter.hasNext()); + try { + iter.next(); + fail("Expected NoSuchElementException when calling next() after exhaustion."); + } catch (NoSuchElementException e) { + // Expected + } + try { + iter.next(); + fail("Expected NoSuchElementException on subsequent calls after exhaustion."); + } catch (NoSuchElementException e) { + // Expected + } + } + + @Test + public void testMergeJoinZipper_MergeSingleElementIterators() throws Exception { + ExceptionIterator iter1 = wrapSync(new TestListIterator<>(Arrays.asList(7))); + ExceptionIterator iter2 = wrapSync(new TestListIterator<>(Arrays.asList(3))); + + AsyncPreFetchIterator async1 = new AsyncPreFetchIterator<>(iter1); + AsyncPreFetchIterator async2 = new AsyncPreFetchIterator<>(iter2); + ZipperAsyncIterator zipper = new ZipperAsyncIterator<>(async1, async2, Integer::compareTo); + ExceptionIterator merged = new AsyncToSyncExceptionIterator<>(zipper); + + List result = consumeIterator(merged); + List expected = Arrays.asList(3, 7); + assertEquals(expected, result); + + async1.close(); + async2.close(); + } + + @Test + public void testMergeJoinZipper_PartialConsumption() throws Exception { + List list1 = Arrays.asList(0, 10, 20); + List list2 = Arrays.asList(5, 15, 25); + List> input = Arrays.asList(new TestListIterator<>(list1), + new TestListIterator<>(list2)); + + ExceptionIterator merged = MergeJoinZipper.buildMergeTree(input, Integer::compareTo); + + // Consume just the first two elements, then stop. + assertTrue(merged.hasNext()); + assertEquals(0, (int) merged.next()); + assertTrue(merged.hasNext()); + assertEquals(5, (int) merged.next()); + + // No further consumption. + } + + @Test + public void testMergeJoinZipper_Random() throws Exception { + // Repeat the random test 5 times + for (int i = 0; i < 5; i++) { + Random rnd = new Random(); + int numLists = 2 + rnd.nextInt(4); // 2 to 5 lists + List> allLists = new ArrayList<>(); + List> input = new ArrayList<>(); + + for (int j = 0; j < numLists; j++) { + int size = rnd.nextInt(10); + List list = rnd.ints(size, 0, 100).boxed().sorted().collect(Collectors.toList()); + allLists.add(list); + input.add(new TestListIterator<>(list)); + } + + ExceptionIterator merged = MergeJoinZipper.buildMergeTree(input, Integer::compareTo); + + List actual = consumeIterator(merged); + List expected = allLists.stream().flatMap(List::stream).sorted().collect(Collectors.toList()); + assertEquals(expected, actual); + } + } +} From 848987561316abe7cc5ec8a2dedb04df45cd250d Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ha=CC=8Avard=20Ottestad?= Date: Mon, 17 Mar 2025 13:05:58 +0100 Subject: [PATCH 20/28] wip --- .../impl/diskimport/SectionCompressor.java | 2 +- .../utils/AsyncIteratorFetcherUnordered.java | 86 ++++++++++++++----- 2 files changed, 64 insertions(+), 24 deletions(-) diff --git a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/hdt/impl/diskimport/SectionCompressor.java b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/hdt/impl/diskimport/SectionCompressor.java index 09e208473..a183821b8 100644 --- a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/hdt/impl/diskimport/SectionCompressor.java +++ b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/hdt/impl/diskimport/SectionCompressor.java @@ -60,7 +60,7 @@ public SectionCompressor(CloseSuppressPath baseFileName, AsyncIteratorFetcher extends AsyncIteratorFetcher { private final Iterator iterator; private final Lock lock = new ReentrantLock(); private boolean end; - Queue[] queue = new Queue[] { new ArrayDeque(BUFFER), new ArrayDeque(BUFFER), new ArrayDeque(BUFFER), + volatile Queue[] queue = new Queue[] { new ArrayDeque(BUFFER), new ArrayDeque(BUFFER), new ArrayDeque(BUFFER), new ArrayDeque(BUFFER), new ArrayDeque(BUFFER), new ArrayDeque(BUFFER), new ArrayDeque(BUFFER), new ArrayDeque(BUFFER), new ArrayDeque(BUFFER), new ArrayDeque(BUFFER), new ArrayDeque(BUFFER), new ArrayDeque(BUFFER), new ArrayDeque(BUFFER), new ArrayDeque(BUFFER), new ArrayDeque(BUFFER), @@ -38,37 +37,78 @@ public E get() { int index = (int) (Thread.currentThread().getId() % queue.length); - // With this approach there is some risk that a queue is filled but - // never emptied. Maybe we should look for another queue to read from - // before filling our own queue? - synchronized (queue[index]) { - E poll = queue[index].poll(); + Queue es = queue[index]; + if (es == null) { + for (Queue eQueue : queue) { + if (eQueue != null) { + synchronized (eQueue) { + E poll = eQueue.poll(); - if (poll != null) { - return poll; + if (poll != null) { + return poll; + } + } + } } + } - synchronized (this) { - poll = queue[index].poll(); - if (poll == null) { - if (iterator.hasNext()) { - poll = iterator.next(); - } - ArrayList objects = new ArrayList<>(BUFFER); + if (es != null) { + // With this approach there is some risk that a queue is filled but + // never emptied. Maybe we should look for another queue to read + // from + // before filling our own queue? + synchronized (es) { + E poll = es.poll(); - for (int i = 0; i < BUFFER && iterator.hasNext(); i++) { - objects.add(iterator.next()); - } + if (poll != null) { + return poll; + } + + synchronized (this) { + es = queue[index]; + if (es != null) { + + poll = es.poll(); + if (poll == null) { + if (iterator.hasNext()) { + poll = iterator.next(); + ArrayList objects = new ArrayList<>(BUFFER); + + for (int i = 0; i < BUFFER && iterator.hasNext(); i++) { + es.add(iterator.next()); + } - queue[index].addAll(objects); + es.addAll(objects); + } + + } + + if (poll == null) { + queue[index] = null; + } else { + return poll; + } + } } + } + } + + for (Queue eQueue : queue) { + if (eQueue != null) { - if (poll == null) { - end = true; + synchronized (eQueue) { + synchronized (this) { + E poll = eQueue.poll(); + + if (poll != null) { + return poll; + } + } } - return poll; } } + end = true; + return null; } From a496a2f373051f1fd33be017d9d9170fa28a4112 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ha=CC=8Avard=20Ottestad?= Date: Mon, 17 Mar 2025 13:06:18 +0100 Subject: [PATCH 21/28] use fastutil for buffered in/out stream --- qendpoint-core/pom.xml | 5 + .../compact/bitmap/MultiRoaringBitmap.java | 10 +- .../compact/sequence/SequenceLog64Map.java | 6 +- .../impl/section/PFCDictionarySectionBig.java | 8 +- .../impl/section/PFCDictionarySectionMap.java | 4 +- .../qendpoint/core/hdt/HDTManagerImpl.java | 5 +- .../qendpoint/core/hdt/impl/HDTImpl.java | 30 +- .../qendpoint/core/hdt/impl/WriteHDTImpl.java | 4 +- .../core/hdt/writer/TripleWriterHDT.java | 9 +- .../core/hdt/writer/TripleWriterNtriples.java | 5 +- .../qendpoint/core/options/ControlInfo.java | 4 +- .../core/options/HDTOptionsFile.java | 10 +- .../core/storage/QEPMapIdSorter.java | 10 +- .../qendpoint/core/storage/TempBuffIn.java | 571 ------------------ .../qendpoint/core/storage/TempBuffOut.java | 170 ------ .../core/triples/impl/BitmapTriplesCat.java | 4 +- .../triples/impl/BitmapTriplesIndexFile.java | 8 +- .../util/LargeFakeDataSetStreamSupplier.java | 4 +- .../qendpoint/core/util/Profiler.java | 8 +- .../qendpoint/core/util/RDFInfo.java | 5 +- .../core/util/io/CloseSuppressPath.java | 8 +- .../qendpoint/core/util/io/IOUtil.java | 8 +- .../core/util/nsd/NamespaceData.java | 8 +- .../bitmap/MultiRoaringBitmapTest.java | 8 +- .../dictionary/impl/kcat/KCatMergerTest.java | 12 +- .../section/WriteDictionarySectionTest.java | 6 +- .../qendpoint/core/rdf/parsers/TarTest.java | 8 +- .../triples/impl/BitmapQuadTriplesTest.java | 10 +- .../LargeFakeDataSetStreamSupplierTest.java | 6 +- .../qendpoint/compiler/SailCompiler.java | 4 +- .../qendpoint/store/EndpointStore.java | 5 +- .../qendpoint/store/MergeRunnable.java | 4 +- .../qendpoint/tools/QEPSearch.java | 4 +- .../qendpoint/utils/BitArrayDisk.java | 5 +- .../qendpoint/utils/FormatUtils.java | 4 +- .../qendpoint/store/MergeRestartTest.java | 14 +- .../qendpoint/store/Utility.java | 4 +- 37 files changed, 138 insertions(+), 860 deletions(-) delete mode 100644 qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/storage/TempBuffIn.java delete mode 100644 qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/storage/TempBuffOut.java diff --git a/qendpoint-core/pom.xml b/qendpoint-core/pom.xml index d0adbad4a..f025669b5 100644 --- a/qendpoint-core/pom.xml +++ b/qendpoint-core/pom.xml @@ -123,5 +123,10 @@ RoaringBitmap ${roaringbitmap.version} + + it.unimi.dsi + fastutil + 8.5.15 + diff --git a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/compact/bitmap/MultiRoaringBitmap.java b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/compact/bitmap/MultiRoaringBitmap.java index 467fc7b5f..a79adb7eb 100644 --- a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/compact/bitmap/MultiRoaringBitmap.java +++ b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/compact/bitmap/MultiRoaringBitmap.java @@ -3,8 +3,8 @@ import com.the_qa_company.qendpoint.core.exceptions.NotImplementedException; import com.the_qa_company.qendpoint.core.hdt.HDTVocabulary; import com.the_qa_company.qendpoint.core.listener.ProgressListener; -import com.the_qa_company.qendpoint.core.storage.TempBuffIn; -import com.the_qa_company.qendpoint.core.storage.TempBuffOut; +import it.unimi.dsi.fastutil.io.FastBufferedInputStream; +import it.unimi.dsi.fastutil.io.FastBufferedOutputStream; import com.the_qa_company.qendpoint.core.util.io.CloseMappedByteBuffer; import com.the_qa_company.qendpoint.core.util.io.Closer; import com.the_qa_company.qendpoint.core.util.io.IOUtil; @@ -69,7 +69,7 @@ public static MultiRoaringBitmap load(InputStream input) throws IOException { * @throws IOException io exception when loading */ public static MultiRoaringBitmap load(Path input) throws IOException { - try (InputStream stream = new TempBuffIn(Files.newInputStream(input))) { + try (InputStream stream = new FastBufferedInputStream(Files.newInputStream(input))) { return load(stream); } } @@ -341,7 +341,7 @@ private void closeStreamBitmap(int layer, int index) throws IOException { int sizeInBytes = handle.serializedSizeInBytes(); outputMax += sizeInBytes + 8 + 8 + 1; - OutputStream os = new TempBuffOut(Channels.newOutputStream(output.position(loc))); + OutputStream os = new FastBufferedOutputStream(Channels.newOutputStream(output.position(loc))); os.write(BLOCK_BITMAP); IOUtil.writeLong(os, sizeInBytes); IOUtil.writeLong(os, layer); @@ -356,7 +356,7 @@ private void closeStreamBitmap(int layer, int index) throws IOException { } public void save(Path output) throws IOException { - try (OutputStream stream = new TempBuffOut(Files.newOutputStream(output))) { + try (OutputStream stream = new FastBufferedOutputStream(Files.newOutputStream(output))) { save(stream); } } diff --git a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/compact/sequence/SequenceLog64Map.java b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/compact/sequence/SequenceLog64Map.java index 809507378..f13252523 100644 --- a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/compact/sequence/SequenceLog64Map.java +++ b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/compact/sequence/SequenceLog64Map.java @@ -25,7 +25,7 @@ import com.the_qa_company.qendpoint.core.exceptions.NotImplementedException; import com.the_qa_company.qendpoint.core.hdt.HDTVocabulary; import com.the_qa_company.qendpoint.core.listener.ProgressListener; -import com.the_qa_company.qendpoint.core.storage.TempBuffIn; +import it.unimi.dsi.fastutil.io.FastBufferedInputStream; import com.the_qa_company.qendpoint.core.util.BitUtil; import com.the_qa_company.qendpoint.core.util.crc.CRC32; import com.the_qa_company.qendpoint.core.util.crc.CRC8; @@ -66,7 +66,7 @@ public class SequenceLog64Map implements Sequence, Closeable { public SequenceLog64Map(File f) throws IOException { // Read from the beginning of the file - this(new CountInputStream(new TempBuffIn(new FileInputStream(f))), f, true); + this(new CountInputStream(new FastBufferedInputStream(new FileInputStream(f))), f, true); } public SequenceLog64Map(CountInputStream in, File f) throws IOException { @@ -163,7 +163,7 @@ private void mapFiles(File f, long base) throws IOException { // FIXME: Bug in the previous code, find what because it should be more // efficient - CountInputStream in = new CountInputStream(new TempBuffIn(new FileInputStream(f))); + CountInputStream in = new CountInputStream(new FastBufferedInputStream(new FileInputStream(f))); IOUtil.skip(in, base + ((numwords - 1) * 8L)); // System.out.println("Last word starts at: "+in.getTotalBytes()); // Read only used bits from last entry (byte aligned, little endian) diff --git a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/dictionary/impl/section/PFCDictionarySectionBig.java b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/dictionary/impl/section/PFCDictionarySectionBig.java index 7dd61e438..6ed1f0892 100644 --- a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/dictionary/impl/section/PFCDictionarySectionBig.java +++ b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/dictionary/impl/section/PFCDictionarySectionBig.java @@ -39,8 +39,8 @@ import com.the_qa_company.qendpoint.core.exceptions.IllegalFormatException; import com.the_qa_company.qendpoint.core.listener.ProgressListener; import com.the_qa_company.qendpoint.core.options.HDTOptions; -import com.the_qa_company.qendpoint.core.storage.TempBuffIn; -import com.the_qa_company.qendpoint.core.storage.TempBuffOut; +import it.unimi.dsi.fastutil.io.FastBufferedInputStream; +import it.unimi.dsi.fastutil.io.FastBufferedOutputStream; import com.the_qa_company.qendpoint.core.util.BitUtil; import com.the_qa_company.qendpoint.core.util.Mutable; import com.the_qa_company.qendpoint.core.util.crc.CRC32; @@ -118,7 +118,7 @@ public void load(Iterator it, long numentries, ProgressL ByteString previousStr = null; try { - try (OutputStream out = new TempBuffOut(new FileOutputStream(file))) { + try (OutputStream out = new FastBufferedOutputStream(new FileOutputStream(file))) { while (it.hasNext()) { ByteString str = ByteString.of(it.next()); @@ -163,7 +163,7 @@ public void load(Iterator it, long numentries, ProgressL byteOut.writeTo(out); } - try (InputStream in = new TempBuffIn(new FileInputStream(file))) { + try (InputStream in = new FastBufferedInputStream(new FileInputStream(file))) { // Read block by block // Read packed data diff --git a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/dictionary/impl/section/PFCDictionarySectionMap.java b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/dictionary/impl/section/PFCDictionarySectionMap.java index 59410ea50..ba5432286 100644 --- a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/dictionary/impl/section/PFCDictionarySectionMap.java +++ b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/dictionary/impl/section/PFCDictionarySectionMap.java @@ -25,7 +25,7 @@ import com.the_qa_company.qendpoint.core.exceptions.IllegalFormatException; import com.the_qa_company.qendpoint.core.exceptions.NotImplementedException; import com.the_qa_company.qendpoint.core.listener.ProgressListener; -import com.the_qa_company.qendpoint.core.storage.TempBuffIn; +import it.unimi.dsi.fastutil.io.FastBufferedInputStream; import com.the_qa_company.qendpoint.core.util.io.BigMappedByteBuffer; import com.the_qa_company.qendpoint.core.compact.integer.VByte; import com.the_qa_company.qendpoint.core.compact.sequence.Sequence; @@ -376,7 +376,7 @@ public void load(Iterator it, long count, ProgressListen @Override public void save(OutputStream output, ProgressListener listener) throws IOException { - InputStream in = new TempBuffIn(new FileInputStream(f)); + InputStream in = new FastBufferedInputStream(new FileInputStream(f)); IOUtil.skip(in, startOffset); IOUtil.copyStream(in, output, endOffset - startOffset); in.close(); diff --git a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/hdt/HDTManagerImpl.java b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/hdt/HDTManagerImpl.java index b9638935e..b4098665a 100644 --- a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/hdt/HDTManagerImpl.java +++ b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/hdt/HDTManagerImpl.java @@ -20,7 +20,7 @@ import com.the_qa_company.qendpoint.core.rdf.RDFParserCallback; import com.the_qa_company.qendpoint.core.rdf.RDFParserFactory; import com.the_qa_company.qendpoint.core.rdf.TripleWriter; -import com.the_qa_company.qendpoint.core.storage.TempBuffOut; +import it.unimi.dsi.fastutil.io.FastBufferedOutputStream; import com.the_qa_company.qendpoint.core.triples.TripleString; import com.the_qa_company.qendpoint.core.util.BitUtil; import com.the_qa_company.qendpoint.core.util.Profiler; @@ -255,7 +255,8 @@ public HDTResult doGenerateHDT(String rdfFileName, String baseURI, RDFNotation r InputStream stream = readIs.is(); try (InputStream is = checksumPath != null ? new CRCInputStream(stream, new CRC32()) : stream; - OutputStream os = new TempBuffOut(Files.newOutputStream(preDownload, openOptions))) { + OutputStream os = new FastBufferedOutputStream( + Files.newOutputStream(preDownload, openOptions))) { IOUtil.copy(is, os, listener, 10_000_000); if (is instanceof CRCInputStream crcIs) { checksum = crcIs.getCRC().getValue(); diff --git a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/hdt/impl/HDTImpl.java b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/hdt/impl/HDTImpl.java index 8ee2af5e7..e27279b5b 100644 --- a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/hdt/impl/HDTImpl.java +++ b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/hdt/impl/HDTImpl.java @@ -52,8 +52,8 @@ import com.the_qa_company.qendpoint.core.options.HDTOptions; import com.the_qa_company.qendpoint.core.options.HDTOptionsKeys; import com.the_qa_company.qendpoint.core.options.HDTSpecification; -import com.the_qa_company.qendpoint.core.storage.TempBuffIn; -import com.the_qa_company.qendpoint.core.storage.TempBuffOut; +import it.unimi.dsi.fastutil.io.FastBufferedInputStream; +import it.unimi.dsi.fastutil.io.FastBufferedOutputStream; import com.the_qa_company.qendpoint.core.triples.DictionaryEntriesDiff; import com.the_qa_company.qendpoint.core.triples.IteratorTripleID; import com.the_qa_company.qendpoint.core.triples.IteratorTripleString; @@ -162,9 +162,9 @@ public void loadFromHDT(InputStream input, ProgressListener listener) throws IOE public void loadFromHDT(String hdtFileName, ProgressListener listener) throws IOException { InputStream in; if (hdtFileName.endsWith(".gz")) { - in = new TempBuffIn(new GZIPInputStream(new FileInputStream(hdtFileName))); + in = new FastBufferedInputStream(new GZIPInputStream(new FileInputStream(hdtFileName))); } else { - in = new CountInputStream(new TempBuffIn(new FileInputStream(hdtFileName))); + in = new CountInputStream(new FastBufferedInputStream(new FileInputStream(hdtFileName))); } loadFromHDT(in, listener); in.close(); @@ -194,8 +194,8 @@ public void mapFromHDT(File f, long offset, ProgressListener listener) throws IO } boolean dumpBinInfo = spec.getBoolean(HDTOptionsKeys.DUMP_BINARY_OFFSETS, false); - try (CountInputStream input = new CountInputStream(new TempBuffIn(new FileInputStream(hdtFileName)), - dumpBinInfo)) { + try (CountInputStream input = new CountInputStream( + new FastBufferedInputStream(new FileInputStream(hdtFileName)), dumpBinInfo)) { input.printIndex("HDT CI"); @@ -258,7 +258,7 @@ public void mapFromHDT(File f, long offset, ProgressListener listener) throws IO */ @Override public void saveToHDT(String fileName, ProgressListener listener) throws IOException { - try (OutputStream out = new TempBuffOut(new FileOutputStream(fileName))) { + try (OutputStream out = new FastBufferedOutputStream(new FileOutputStream(fileName))) { // OutputStream out = new GZIPOutputStream(new // BufferedOutputStream(new FileOutputStream(fileName))); saveToHDT(out, listener); @@ -512,7 +512,7 @@ public void loadOrCreateIndex(ProgressListener listener, HDTOptions spec) throws } CountInputStream in = null; try { - in = new CountInputStream(new TempBuffIn(new FileInputStream(ff))); + in = new CountInputStream(new FastBufferedInputStream(new FileInputStream(ff))); ci.load(in); if (isMapped) { triples.mapIndex(in, new File(indexName), ci, listener); @@ -532,7 +532,7 @@ public void loadOrCreateIndex(ProgressListener listener, HDTOptions spec) throws if (this.hdtFileName != null) { OutputStream out = null; try { - out = new TempBuffOut(new FileOutputStream(versionName)); + out = new FastBufferedOutputStream(new FileOutputStream(versionName)); ci.clear(); triples.saveIndex(out, ci, listener); out.close(); @@ -613,7 +613,7 @@ public void cat(String location, HDT hdt1, HDT hdt2, ProgressListener listener, // map the generated dictionary FourSectionDictionaryBig dictionary; try (CountInputStream fis = new CountInputStream( - new TempBuffIn(new FileInputStream(location + "dictionary")))) { + new FastBufferedInputStream(new FileInputStream(location + "dictionary")))) { dictionary = new FourSectionDictionaryBig(new HDTSpecification()); fis.mark(1024); ci2.load(fis); @@ -657,7 +657,8 @@ public void cat(String location, HDT hdt1, HDT hdt2, ProgressListener listener, Files.delete(Paths.get(location + "O2" + "Types")); // map the triples - try (CountInputStream fis2 = new CountInputStream(new TempBuffIn(new FileInputStream(location + "triples")))) { + try (CountInputStream fis2 = new CountInputStream( + new FastBufferedInputStream(new FileInputStream(location + "triples")))) { ControlInfo ci2 = new ControlInformation(); ci2.clear(); fis2.mark(1024); @@ -696,7 +697,7 @@ public void catCustom(String location, HDT hdt1, HDT hdt2, ProgressListener list // map the generated dictionary ControlInfo ci2 = new ControlInformation(); try (CountInputStream fis = new CountInputStream( - new TempBuffIn(new FileInputStream(location + "dictionary")))) { + new FastBufferedInputStream(new FileInputStream(location + "dictionary")))) { HDTSpecification spec = new HDTSpecification(); spec.set(HDTOptionsKeys.TEMP_DICTIONARY_IMPL_KEY, HDTOptionsKeys.TEMP_DICTIONARY_IMPL_VALUE_MULT_HASH); spec.set(HDTOptionsKeys.DICTIONARY_TYPE_KEY, HDTOptionsKeys.DICTIONARY_TYPE_VALUE_MULTI_OBJECTS); @@ -758,7 +759,8 @@ public void catCustom(String location, HDT hdt1, HDT hdt2, ProgressListener list Files.delete(Paths.get(location + "O2")); Files.delete(Paths.get(location + "O2" + "Types")); // map the triples - try (CountInputStream fis2 = new CountInputStream(new TempBuffIn(new FileInputStream(location + "triples")))) { + try (CountInputStream fis2 = new CountInputStream( + new FastBufferedInputStream(new FileInputStream(location + "triples")))) { ControlInformation ci2 = new ControlInformation(); ci2.clear(); fis2.mark(1024); @@ -815,7 +817,7 @@ public void diffBit(String location, HDT hdt, Bitmap deleteBitmap, ProgressListe ControlInfo ci2 = new ControlInformation(); try (CountInputStream fis = new CountInputStream( - new TempBuffIn(new FileInputStream(location + "dictionary")))) { + new FastBufferedInputStream(new FileInputStream(location + "dictionary")))) { fis.mark(1024); ci2.load(fis); fis.reset(); diff --git a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/hdt/impl/WriteHDTImpl.java b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/hdt/impl/WriteHDTImpl.java index dbccae45e..2f174bf00 100644 --- a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/hdt/impl/WriteHDTImpl.java +++ b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/hdt/impl/WriteHDTImpl.java @@ -8,7 +8,7 @@ import com.the_qa_company.qendpoint.core.header.HeaderPrivate; import com.the_qa_company.qendpoint.core.listener.ProgressListener; import com.the_qa_company.qendpoint.core.options.HDTOptions; -import com.the_qa_company.qendpoint.core.storage.TempBuffOut; +import it.unimi.dsi.fastutil.io.FastBufferedOutputStream; import com.the_qa_company.qendpoint.core.triples.IteratorTripleString; import com.the_qa_company.qendpoint.core.triples.TriplesPrivate; import com.the_qa_company.qendpoint.core.triples.impl.WriteBitmapTriples; @@ -88,7 +88,7 @@ public void loadOrCreateIndex(ProgressListener listener, HDTOptions disk) { @Override public void saveToHDT(String fileName, ProgressListener listener) throws IOException { - try (OutputStream out = new TempBuffOut(Files.newOutputStream(Path.of(fileName)))) { + try (OutputStream out = new FastBufferedOutputStream(Files.newOutputStream(Path.of(fileName)))) { saveToHDT(out, listener); } } diff --git a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/hdt/writer/TripleWriterHDT.java b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/hdt/writer/TripleWriterHDT.java index 27c1443a5..f51910e37 100644 --- a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/hdt/writer/TripleWriterHDT.java +++ b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/hdt/writer/TripleWriterHDT.java @@ -18,7 +18,7 @@ import com.the_qa_company.qendpoint.core.listener.ProgressListener; import com.the_qa_company.qendpoint.core.options.HDTOptions; import com.the_qa_company.qendpoint.core.rdf.TripleWriter; -import com.the_qa_company.qendpoint.core.storage.TempBuffOut; +import it.unimi.dsi.fastutil.io.FastBufferedOutputStream; import com.the_qa_company.qendpoint.core.triples.TempTriples; import com.the_qa_company.qendpoint.core.triples.TripleString; import com.the_qa_company.qendpoint.core.util.StopWatch; @@ -40,9 +40,10 @@ public TripleWriterHDT(String baseUri, HDTOptions spec, String outFile, boolean this.baseUri = baseUri; this.spec = spec; if (compress) { - this.out = new TempBuffOut(new GZIPOutputStream(new TempBuffOut(new FileOutputStream(outFile)))); + this.out = new FastBufferedOutputStream( + new GZIPOutputStream(new FastBufferedOutputStream(new FileOutputStream(outFile)))); } else { - this.out = new TempBuffOut(new FileOutputStream(outFile)); + this.out = new FastBufferedOutputStream(new FileOutputStream(outFile), 4 * 1024 * 1024); } close = true; init(); @@ -51,7 +52,7 @@ public TripleWriterHDT(String baseUri, HDTOptions spec, String outFile, boolean public TripleWriterHDT(String baseUri, HDTOptions spec, OutputStream out) { this.baseUri = baseUri; this.spec = spec; - this.out = new TempBuffOut(out); + this.out = new FastBufferedOutputStream(out); init(); } diff --git a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/hdt/writer/TripleWriterNtriples.java b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/hdt/writer/TripleWriterNtriples.java index c0934124b..32ef26d05 100644 --- a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/hdt/writer/TripleWriterNtriples.java +++ b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/hdt/writer/TripleWriterNtriples.java @@ -11,7 +11,7 @@ import java.util.zip.GZIPOutputStream; import com.the_qa_company.qendpoint.core.rdf.TripleWriter; -import com.the_qa_company.qendpoint.core.storage.TempBuffOut; +import it.unimi.dsi.fastutil.io.FastBufferedOutputStream; import com.the_qa_company.qendpoint.core.triples.TripleString; public class TripleWriterNtriples implements TripleWriter { @@ -21,7 +21,8 @@ public class TripleWriterNtriples implements TripleWriter { public TripleWriterNtriples(String outFile, boolean compress) throws IOException { if (compress) { - this.out = new OutputStreamWriter(new GZIPOutputStream(new TempBuffOut(new FileOutputStream(outFile)))); + this.out = new OutputStreamWriter( + new GZIPOutputStream(new FastBufferedOutputStream(new FileOutputStream(outFile)))); } else { this.out = new BufferedWriter(new FileWriter(outFile)); } diff --git a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/options/ControlInfo.java b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/options/ControlInfo.java index 2fb603c72..cb6ab7dff 100644 --- a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/options/ControlInfo.java +++ b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/options/ControlInfo.java @@ -19,7 +19,7 @@ package com.the_qa_company.qendpoint.core.options; -import com.the_qa_company.qendpoint.core.storage.TempBuffOut; +import it.unimi.dsi.fastutil.io.FastBufferedOutputStream; import java.io.BufferedOutputStream; import java.io.IOException; @@ -45,7 +45,7 @@ enum Type { void setFormat(String format); default void save(Path filename) throws IOException { - try (OutputStream os = new TempBuffOut(Files.newOutputStream(filename))) { + try (OutputStream os = new FastBufferedOutputStream(Files.newOutputStream(filename))) { save(os); } } diff --git a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/options/HDTOptionsFile.java b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/options/HDTOptionsFile.java index 4c83f8ef2..cda54ffce 100644 --- a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/options/HDTOptionsFile.java +++ b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/options/HDTOptionsFile.java @@ -2,8 +2,8 @@ import com.the_qa_company.qendpoint.core.compact.integer.VByte; import com.the_qa_company.qendpoint.core.listener.ProgressListener; -import com.the_qa_company.qendpoint.core.storage.TempBuffIn; -import com.the_qa_company.qendpoint.core.storage.TempBuffOut; +import it.unimi.dsi.fastutil.io.FastBufferedInputStream; +import it.unimi.dsi.fastutil.io.FastBufferedOutputStream; import com.the_qa_company.qendpoint.core.util.crc.CRC32; import com.the_qa_company.qendpoint.core.util.crc.CRCInputStream; import com.the_qa_company.qendpoint.core.util.crc.CRCOutputStream; @@ -39,7 +39,8 @@ public void sync() throws IOException { return; } ProgressListener l = ProgressListener.ignore(); - try (CRCInputStream is = new CRCInputStream(new TempBuffIn(Files.newInputStream(location)), new CRC32())) { + try (CRCInputStream is = new CRCInputStream(new FastBufferedInputStream(Files.newInputStream(location)), + new CRC32())) { if (IOUtil.readLong(is) != MAGIC) throw new IOException("Can't read HDTOptions file: Bad magic"); @@ -60,7 +61,8 @@ public void sync() throws IOException { public void save() throws IOException { ProgressListener l = ProgressListener.ignore(); - try (CRCOutputStream os = new CRCOutputStream(new TempBuffOut(Files.newOutputStream(location)), new CRC32())) { + try (CRCOutputStream os = new CRCOutputStream(new FastBufferedOutputStream(Files.newOutputStream(location)), + new CRC32())) { IOUtil.writeLong(os, MAGIC); Set keys = options.getKeys(); VByte.encode(os, keys.size()); diff --git a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/storage/QEPMapIdSorter.java b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/storage/QEPMapIdSorter.java index 3d8b4509a..a4b83df6a 100644 --- a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/storage/QEPMapIdSorter.java +++ b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/storage/QEPMapIdSorter.java @@ -14,6 +14,8 @@ import com.the_qa_company.qendpoint.core.util.disk.LongArray; import com.the_qa_company.qendpoint.core.util.io.CloseSuppressPath; import com.the_qa_company.qendpoint.core.util.io.Closer; +import it.unimi.dsi.fastutil.io.FastBufferedInputStream; +import it.unimi.dsi.fastutil.io.FastBufferedOutputStream; import java.io.BufferedInputStream; import java.io.BufferedOutputStream; @@ -94,7 +96,7 @@ public void sort() throws IOException { CloseSuppressPath output = merger.waitResult().orElse(null); if (output != null) { - try (InputStream stream = new TempBuffIn(Files.newInputStream(output))) { + try (InputStream stream = new FastBufferedInputStream(Files.newInputStream(output))) { QEPMapReader reader = new QEPMapReader(stream); long index = 0; @@ -156,7 +158,7 @@ private record Merger(long chunkSize) implements KWayMerger.KWayMergerImpl flux, CloseSuppressPath output) throws KWayMerger.KWayMergerException { - try (OutputStream stream = new TempBuffOut(Files.newOutputStream(output))) { + try (OutputStream stream = new FastBufferedOutputStream(Files.newOutputStream(output))) { QEPMapIds ids; List idList = new ArrayList<>(); @@ -199,7 +201,7 @@ public void mergeChunks(List inputs, CloseSuppressPath output InputStream[] pathInput = new InputStream[inputs.size()]; for (int i = 0; i < pathInput.length; i++) { - pathInput[i] = new TempBuffIn(Files.newInputStream(inputs.get(i))); + pathInput[i] = new FastBufferedInputStream(Files.newInputStream(inputs.get(i))); } try { @@ -207,7 +209,7 @@ public void mergeChunks(List inputs, CloseSuppressPath output ExceptionIterator tree = MergeExceptionIterator .buildOfTree(QEPMapReader::new, Arrays.asList(pathInput), 0, inputs.size()); - try (OutputStream stream = new TempBuffOut(Files.newOutputStream(output))) { + try (OutputStream stream = new FastBufferedOutputStream(Files.newOutputStream(output))) { while (tree.hasNext()) { QEPMapIds ids = tree.next(); VByte.encode(stream, ids.origin()); diff --git a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/storage/TempBuffIn.java b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/storage/TempBuffIn.java deleted file mode 100644 index 1050e92bb..000000000 --- a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/storage/TempBuffIn.java +++ /dev/null @@ -1,571 +0,0 @@ -/* - * Copyright (c) 1994, 2024, Oracle and/or its affiliates. All rights reserved. - * DO NOT ALTER OR REMOVE COPYRIGHT NOTICES OR THIS FILE HEADER. This code is - * free software; you can redistribute it and/or modify it under the terms of - * the GNU General Public License version 2 only, as published by the Free - * Software Foundation. Oracle designates this particular file as subject to the - * "Classpath" exception as provided by Oracle in the LICENSE file that - * accompanied this code. This code is distributed in the hope that it will be - * useful, but WITHOUT ANY WARRANTY; without even the implied warranty of - * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the GNU General - * Public License version 2 for more details (a copy is included in the LICENSE - * file that accompanied this code). You should have received a copy of the GNU - * General Public License version 2 along with this work; if not, write to the - * Free Software Foundation, Inc., 51 Franklin St, Fifth Floor, Boston, MA - * 02110-1301 USA. Please contact Oracle, 500 Oracle Parkway, Redwood Shores, CA - * 94065 USA or visit www.oracle.com if you need additional information or have - * any questions. - */ - -package com.the_qa_company.qendpoint.core.storage; - -import jdk.internal.misc.Unsafe; -import jdk.internal.util.ArraysSupport; - -import java.io.ByteArrayOutputStream; -import java.io.FileOutputStream; -import java.io.FilterInputStream; -import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; -import java.io.PipedOutputStream; -import java.util.Arrays; -import java.util.Objects; - -/** - * A {@code TempBuffIn} adds functionality to another input stream-namely, the - * ability to buffer the input and to support the {@code mark} and {@code reset} - * methods. When the {@code TempBuffIn} is created, an internal buffer array is - * created. As bytes from the stream are read or skipped, the internal buffer is - * refilled as necessary from the contained input stream, many bytes at a time. - * The {@code mark} operation remembers a point in the input stream and the - * {@code reset} operation causes all the bytes read since the most recent - * {@code mark} operation to be reread before new bytes are taken from the - * contained input stream. - * - * @author Arthur van Hoff - * @since 1.0 - */ -public class TempBuffIn extends FilterInputStream { - - private static final int DEFAULT_BUFFER_SIZE = 8192; - - private static final byte[] EMPTY = new byte[0]; - - /** - * As this class is used early during bootstrap, it's motivated to use - * Unsafe.compareAndSetReference instead of AtomicReferenceFieldUpdater (or - * VarHandles) to reduce dependencies and improve startup time. - */ - private static final Unsafe U = Unsafe.getUnsafe(); - - private static final long BUF_OFFSET = U.objectFieldOffset(TempBuffIn.class, "buf"); - - // initial buffer size (DEFAULT_BUFFER_SIZE or size specified to - // constructor) - private final int initialSize; - - /** - * The internal buffer array where the data is stored. When necessary, it - * may be replaced by another array of a different size. - */ - /* - * We null this out with a CAS on close(), which is necessary since closes - * can be asynchronous. We use nullness of buf[] as primary indicator that - * this stream is closed. (The "in" field is also nulled out on close.) - */ - protected volatile byte[] buf; - - /** - * The index one greater than the index of the last valid byte in the - * buffer. This value is always in the range {@code 0} through - * {@code buf.length}; elements {@code buf[0]} through {@code buf[count-1]} - * contain buffered input data obtained from the underlying input stream. - */ - protected int count; - - /** - * The current position in the buffer. This is the index of the next byte to - * be read from the {@code buf} array. - *

- * This value is always in the range {@code 0} through {@code count}. If it - * is less than {@code count}, then {@code buf[pos]} is the next byte to be - * supplied as input; if it is equal to {@code count}, then the next - * {@code read} or {@code skip} operation will require more bytes to be read - * from the contained input stream. - */ - protected int pos; - - /** - * The value of the {@code pos} field at the time the last {@code mark} - * method was called. - *

- * This value is always in the range {@code -1} through {@code pos}. If - * there is no marked position in the input stream, this field is - * {@code -1}. If there is a marked position in the input stream, then - * {@code buf[markpos]} is the first byte to be supplied as input after a - * {@code reset} operation. If {@code markpos} is not {@code -1}, then all - * bytes from positions {@code buf[markpos]} through {@code buf[pos-1]} must - * remain in the buffer array (though they may be moved to another place in - * the buffer array, with suitable adjustments to the values of - * {@code count}, {@code pos}, and {@code markpos}); they may not be - * discarded unless and until the difference between {@code pos} and - * {@code markpos} exceeds {@code marklimit}. - */ - protected int markpos = -1; - - /** - * The maximum read ahead allowed after a call to the {@code mark} method - * before subsequent calls to the {@code reset} method fail. Whenever the - * difference between {@code pos} and {@code markpos} exceeds - * {@code marklimit}, then the mark may be dropped by setting - * {@code markpos} to {@code -1}. - */ - protected int marklimit; - - /** - * Check to make sure that underlying input stream has not been nulled out - * due to close; if not return it; - */ - private InputStream getInIfOpen() throws IOException { - InputStream input = in; - if (input == null) { - throw new IOException("Stream closed"); - } - return input; - } - - /** - * Returns the internal buffer, optionally allocating it if empty. - * - * @param allocateIfEmpty true to allocate if empty - * @throws IOException if the stream is closed (buf is null) - */ - private byte[] getBufIfOpen(boolean allocateIfEmpty) throws IOException { - byte[] buffer = buf; - if (allocateIfEmpty && buffer == EMPTY) { - buffer = new byte[initialSize]; - if (!U.compareAndSetReference(this, BUF_OFFSET, EMPTY, buffer)) { - // re-read buf - buffer = buf; - } - } - if (buffer == null) { - throw new IOException("Stream closed"); - } - return buffer; - } - - /** - * Returns the internal buffer, allocating it if empty. - * - * @throws IOException if the stream is closed (buf is null) - */ - private byte[] getBufIfOpen() throws IOException { - return getBufIfOpen(true); - } - - /** - * Throws IOException if the stream is closed (buf is null). - */ - private void ensureOpen() throws IOException { - if (buf == null) { - throw new IOException("Stream closed"); - } - } - - /** - * Creates a {@code TempBuffIn} and saves its argument, the input stream - * {@code in}, for later use. An internal buffer array is created and stored - * in {@code buf}. - * - * @param in the underlying input stream. - */ - public TempBuffIn(InputStream in) { - this(in, DEFAULT_BUFFER_SIZE); - } - - /** - * Creates a {@code TempBuffIn} with the specified buffer size, and saves - * its argument, the input stream {@code in}, for later use. An internal - * buffer array of length {@code size} is created and stored in {@code buf}. - * - * @param in the underlying input stream. - * @param size the buffer size. - * @throws IllegalArgumentException if {@code size <= 0}. - */ - public TempBuffIn(InputStream in, int size) { - super(in); - if (size <= 0) { - throw new IllegalArgumentException("Buffer size <= 0"); - } - initialSize = size; - - buf = new byte[size]; - - } - - /** - * Fills the buffer with more data, taking into account shuffling and other - * tricks for dealing with marks. Assumes that it is being called by a - * locked method. This method also assumes that all data has already been - * read in, hence pos > count. - */ - private void fill() throws IOException { - byte[] buffer = getBufIfOpen(); - if (markpos == -1) { - pos = 0; /* no mark: throw away the buffer */ - } else if (pos >= buffer.length) { /* no room left in buffer */ - if (markpos > 0) { /* can throw away early part of the buffer */ - int sz = pos - markpos; - System.arraycopy(buffer, markpos, buffer, 0, sz); - pos = sz; - markpos = 0; - } else if (buffer.length >= marklimit) { - markpos = -1; /* buffer got too big, invalidate mark */ - pos = 0; /* drop buffer contents */ - } else { /* grow buffer */ - int nsz = ArraysSupport.newLength(pos, 1, /* minimum growth */ - pos /* preferred growth */); - if (nsz > marklimit) { - nsz = marklimit; - } - byte[] nbuf = new byte[nsz]; - System.arraycopy(buffer, 0, nbuf, 0, pos); - if (!U.compareAndSetReference(this, BUF_OFFSET, buffer, nbuf)) { - // Can't replace buf if there was an async close. - // Note: This would need to be changed if fill() - // is ever made accessible to multiple threads. - // But for now, the only way CAS can fail is via close. - // assert buf == null; - throw new IOException("Stream closed"); - } - buffer = nbuf; - } - } - count = pos; - int n = getInIfOpen().read(buffer, pos, buffer.length - pos); - if (n > 0) { - count = n + pos; - } - } - - /** - * See the general contract of the {@code read} method of - * {@code InputStream}. - * - * @return the next byte of data, or {@code -1} if the end of the stream is - * reached. - * @throws IOException if this input stream has been closed by invoking its - * {@link #close()} method, or an I/O error occurs. - */ - public int read() throws IOException { - - return implRead(); - - } - - private int implRead() throws IOException { - if (pos >= count) { - fill(); - if (pos >= count) { - return -1; - } - } - return getBufIfOpen()[pos++] & 0xff; - } - - /** - * Read bytes into a portion of an array, reading from the underlying stream - * at most once if necessary. - */ - private int read1(byte[] b, int off, int len) throws IOException { - int avail = count - pos; - if (avail <= 0) { - /* - * If the requested length is at least as large as the buffer, and - * if there is no mark/reset activity, do not bother to copy the - * bytes into the local buffer. In this way buffered streams will - * cascade harmlessly. - */ - int size = Math.max(getBufIfOpen(false).length, initialSize); - if (len >= size && markpos == -1) { - return getInIfOpen().read(b, off, len); - } - fill(); - avail = count - pos; - if (avail <= 0) { - return -1; - } - } - int cnt = (avail < len) ? avail : len; - System.arraycopy(getBufIfOpen(), pos, b, off, cnt); - pos += cnt; - return cnt; - } - - /** - * Reads bytes from this byte-input stream into the specified byte array, - * starting at the given offset. - *

- * This method implements the general contract of the corresponding - * {@link InputStream#read(byte[], int, int) read} method of the - * {@link InputStream} class. As an additional convenience, it attempts to - * read as many bytes as possible by repeatedly invoking the {@code read} - * method of the underlying stream. This iterated {@code read} continues - * until one of the following conditions becomes true: - *

    - *
  • The specified number of bytes have been read, - *
  • The {@code read} method of the underlying stream returns {@code -1}, - * indicating end-of-file, or - *
  • The {@code available} method of the underlying stream returns zero, - * indicating that further input requests would block. - *
- * If the first {@code read} on the underlying stream returns {@code -1} to - * indicate end-of-file then this method returns {@code -1}. Otherwise, this - * method returns the number of bytes actually read. - *

- * Subclasses of this class are encouraged, but not required, to attempt to - * read as many bytes as possible in the same fashion. - * - * @param b destination buffer. - * @param off offset at which to start storing bytes. - * @param len maximum number of bytes to read. - * @return the number of bytes read, or {@code -1} if the end of the stream - * has been reached. - * @throws IOException if this input stream has been closed by - * invoking its {@link #close()} method, - * or an I/O error occurs. - * @throws IndexOutOfBoundsException {@inheritDoc} - */ - public int read(byte[] b, int off, int len) throws IOException { - - return implRead(b, off, len); - - } - - private int implRead(byte[] b, int off, int len) throws IOException { - ensureOpen(); - if ((off | len | (off + len) | (b.length - (off + len))) < 0) { - throw new IndexOutOfBoundsException(); - } else if (len == 0) { - return 0; - } - - int n = 0; - for (;;) { - int nread = read1(b, off + n, len - n); - if (nread <= 0) { - return (n == 0) ? nread : n; - } - n += nread; - if (n >= len) { - return n; - } - // if not closed but no bytes available, return - InputStream input = in; - if (input != null && input.available() <= 0) { - return n; - } - } - } - - /** - * See the general contract of the {@code skip} method of - * {@code InputStream}. - * - * @throws IOException if this input stream has been closed by invoking its - * {@link #close()} method, {@code in.skip(n)} throws an - * IOException, or an I/O error occurs. - */ - public long skip(long n) throws IOException { - - return implSkip(n); - - } - - private long implSkip(long n) throws IOException { - ensureOpen(); - if (n <= 0) { - return 0; - } - long avail = count - pos; - - if (avail <= 0) { - // If no mark position set then don't keep in buffer - if (markpos == -1) { - return getInIfOpen().skip(n); - } - - // Fill in buffer to save bytes for reset - fill(); - avail = count - pos; - if (avail <= 0) { - return 0; - } - } - - long skipped = (avail < n) ? avail : n; - pos += (int) skipped; - return skipped; - } - - /** - * Returns an estimate of the number of bytes that can be read (or skipped - * over) from this input stream without blocking by the next invocation of a - * method for this input stream. The next invocation might be the same - * thread or another thread. A single read or skip of this many bytes will - * not block, but may read or skip fewer bytes. - *

- * This method returns the sum of the number of bytes remaining to be read - * in the buffer ({@code count - pos}) and the result of calling the - * {@link java.io.FilterInputStream#in in}{@code .available()}. - * - * @return an estimate of the number of bytes that can be read (or skipped - * over) from this input stream without blocking. - * @throws IOException if this input stream has been closed by invoking its - * {@link #close()} method, or an I/O error occurs. - */ - public int available() throws IOException { - - return implAvailable(); - - } - - private int implAvailable() throws IOException { - int n = count - pos; - int avail = getInIfOpen().available(); - return n > (Integer.MAX_VALUE - avail) ? Integer.MAX_VALUE : n + avail; - } - - /** - * See the general contract of the {@code mark} method of - * {@code InputStream}. - * - * @param readlimit the maximum limit of bytes that can be read before the - * mark position becomes invalid. - */ - public void mark(int readlimit) { - - implMark(readlimit); - - } - - private void implMark(int readlimit) { - marklimit = readlimit; - markpos = pos; - } - - /** - * See the general contract of the {@code reset} method of - * {@code InputStream}. - *

- * If {@code markpos} is {@code -1} (no mark has been set or the mark has - * been invalidated), an {@code IOException} is thrown. Otherwise, - * {@code pos} is set equal to {@code markpos}. - * - * @throws IOException if this stream has not been marked or, if the mark - * has been invalidated, or the stream has been closed - * by invoking its {@link #close()} method, or an I/O - * error occurs. - */ - public void reset() throws IOException { - - implReset(); - - } - - private void implReset() throws IOException { - ensureOpen(); - if (markpos < 0) { - throw new IOException("Resetting to invalid mark"); - } - pos = markpos; - } - - /** - * Tests if this input stream supports the {@code mark} and {@code reset} - * methods. The {@code markSupported} method of {@code TempBuffIn} returns - * {@code true}. - * - * @return a {@code boolean} indicating if this stream type supports the - * {@code mark} and {@code reset} methods. - */ - public boolean markSupported() { - return true; - } - - /** - * Closes this input stream and releases any system resources associated - * with the stream. Once the stream has been closed, further read(), - * available(), reset(), or skip() invocations will throw an IOException. - * Closing a previously closed stream has no effect. - * - * @throws IOException if an I/O error occurs. - */ - public void close() throws IOException { - byte[] buffer; - while ((buffer = buf) != null) { - if (U.compareAndSetReference(this, BUF_OFFSET, buffer, null)) { - InputStream input = in; - in = null; - if (input != null) { - input.close(); - } - return; - } - // Else retry in case a new buf was CASed in fill() - } - } - - @Override - public long transferTo(OutputStream out) throws IOException { - Objects.requireNonNull(out, "out"); - - return implTransferTo(out); - - } - - private long implTransferTo(OutputStream out) throws IOException { - if (getClass() == TempBuffIn.class && markpos == -1) { - int avail = count - pos; - if (avail > 0) { - if (isTrusted(out)) { - out.write(getBufIfOpen(), pos, avail); - } else { - // Prevent poisoning and leaking of buf - byte[] buffer = Arrays.copyOfRange(getBufIfOpen(), pos, count); - out.write(buffer); - } - pos = count; - } - try { - return Math.addExact(avail, getInIfOpen().transferTo(out)); - } catch (ArithmeticException ignore) { - return Long.MAX_VALUE; - } - } else { - return super.transferTo(out); - } - } - - /** - * Returns true if this class satisfies the following conditions: - *

    - *
  • does not retain a reference to the {@code byte[]}
  • - *
  • does not leak a reference to the {@code byte[]} to non-trusted - * classes
  • - *
  • does not modify the contents of the {@code byte[]}
  • - *
  • {@code write()} method does not read the contents outside of the - * offset/length bounds
  • - *
- * - * @return true if this class is trusted - */ - private static boolean isTrusted(OutputStream os) { - var clazz = os.getClass(); - return clazz == ByteArrayOutputStream.class || clazz == FileOutputStream.class - || clazz == PipedOutputStream.class; - } - -} diff --git a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/storage/TempBuffOut.java b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/storage/TempBuffOut.java deleted file mode 100644 index 5a20781ef..000000000 --- a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/storage/TempBuffOut.java +++ /dev/null @@ -1,170 +0,0 @@ -package com.the_qa_company.qendpoint.core.storage; - -import java.io.FilterOutputStream; -import java.io.IOException; -import java.io.OutputStream; -import java.util.Arrays; - -/** - * The class implements a buffered output stream. By setting up such an output - * stream, an application can write bytes to the underlying output stream - * without necessarily causing a call to the underlying system for each byte - * written. - * - * @author Arthur van Hoff - * @since 1.0 - */ -public class TempBuffOut extends FilterOutputStream { - private static final int DEFAULT_INITIAL_BUFFER_SIZE = 512; - private static final int DEFAULT_MAX_BUFFER_SIZE = 8192; - - /** - * The internal buffer where data is stored. - */ - protected byte[] buf; - - /** - * The number of valid bytes in the buffer. This value is always in the - * range {@code 0} through {@code buf.length}; elements {@code buf[0]} - * through {@code buf[count-1]} contain valid byte data. - */ - protected int count; - - /** - * Max size of the internal buffer. - */ - private final int maxBufSize; - - /** - * Returns the buffer size to use when no output buffer size specified. - */ - private static int initialBufferSize() { - return DEFAULT_MAX_BUFFER_SIZE; - } - - /** - * Creates a new buffered output stream. - */ - private TempBuffOut(OutputStream out, int initialSize, int maxSize) { - super(out); - - if (initialSize <= 0) { - throw new IllegalArgumentException("Buffer size <= 0"); - } - - this.buf = new byte[maxSize]; - - this.maxBufSize = maxSize; - } - - /** - * Creates a new buffered output stream to write data to the specified - * underlying output stream. - * - * @param out the underlying output stream. - */ - public TempBuffOut(OutputStream out) { - this(out, initialBufferSize(), DEFAULT_MAX_BUFFER_SIZE); - } - - /** - * Creates a new buffered output stream to write data to the specified - * underlying output stream with the specified buffer size. - * - * @param out the underlying output stream. - * @param size the buffer size. - * @throws IllegalArgumentException if size <= 0. - */ - public TempBuffOut(OutputStream out, int size) { - this(out, size, size); - } - - /** Flush the internal buffer */ - private void flushBuffer() throws IOException { - if (count > 0) { - out.write(buf, 0, count); - count = 0; - } - } - - /** - * Writes the specified byte to this buffered output stream. - * - * @param b the byte to be written. - * @throws IOException if an I/O error occurs. - */ - @Override - public void write(int b) throws IOException { - - implWrite(b); - - } - - private void implWrite(int b) throws IOException { - if (count >= buf.length) { - flushBuffer(); - } - buf[count++] = (byte) b; - } - - /** - * Writes {@code len} bytes from the specified byte array starting at offset - * {@code off} to this buffered output stream. - *

- * Ordinarily this method stores bytes from the given array into this - * stream's buffer, flushing the buffer to the underlying output stream as - * needed. If the requested length is at least as large as this stream's - * buffer, however, then this method will flush the buffer and write the - * bytes directly to the underlying output stream. Thus redundant - * {@code TempBuffOut}s will not copy data unnecessarily. - * - * @param b the data. - * @param off the start offset in the data. - * @param len the number of bytes to write. - * @throws IOException if an I/O error occurs. - * @throws IndexOutOfBoundsException {@inheritDoc} - */ - @Override - public void write(byte[] b, int off, int len) throws IOException { - - implWrite(b, off, len); - - } - - private void implWrite(byte[] b, int off, int len) throws IOException { - if (len >= maxBufSize) { - /* - * If the request length exceeds the max size of the output buffer, - * flush the output buffer and then write the data directly. In this - * way buffered streams will cascade harmlessly. - */ - flushBuffer(); - out.write(b, off, len); - return; - } - if (len > buf.length - count) { - flushBuffer(); - } - System.arraycopy(b, off, buf, count, len); - count += len; - } - - /** - * Flushes this buffered output stream. This forces any buffered output - * bytes to be written out to the underlying output stream. - * - * @throws IOException if an I/O error occurs. - * @see java.io.FilterOutputStream#out - */ - @Override - public void flush() throws IOException { - - implFlush(); - - } - - private void implFlush() throws IOException { - flushBuffer(); - out.flush(); - } -} diff --git a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/triples/impl/BitmapTriplesCat.java b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/triples/impl/BitmapTriplesCat.java index 4b166783e..00c5ce629 100644 --- a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/triples/impl/BitmapTriplesCat.java +++ b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/triples/impl/BitmapTriplesCat.java @@ -26,7 +26,7 @@ import com.the_qa_company.qendpoint.core.listener.ProgressListener; import com.the_qa_company.qendpoint.core.options.ControlInfo; import com.the_qa_company.qendpoint.core.options.ControlInformation; -import com.the_qa_company.qendpoint.core.storage.TempBuffOut; +import it.unimi.dsi.fastutil.io.FastBufferedOutputStream; import com.the_qa_company.qendpoint.core.triples.IteratorTripleID; import com.the_qa_company.qendpoint.core.triples.TripleID; import com.the_qa_company.qendpoint.core.util.BitUtil; @@ -120,7 +120,7 @@ public void cat(IteratorTripleID it, ProgressListener listener) throws IOExcepti vectorY.aggressiveTrimToSize(); vectorZ.trimToSize(); - try (OutputStream bos = new TempBuffOut(new FileOutputStream(location + "triples"))) { + try (OutputStream bos = new FastBufferedOutputStream(new FileOutputStream(location + "triples"))) { ControlInfo ci = new ControlInformation(); ci.setType(ControlInfo.Type.TRIPLES); ci.setFormat(HDTVocabulary.TRIPLES_TYPE_BITMAP); diff --git a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/triples/impl/BitmapTriplesIndexFile.java b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/triples/impl/BitmapTriplesIndexFile.java index fe7abc73e..d672b6b9c 100644 --- a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/triples/impl/BitmapTriplesIndexFile.java +++ b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/triples/impl/BitmapTriplesIndexFile.java @@ -19,8 +19,8 @@ import com.the_qa_company.qendpoint.core.listener.ProgressListener; import com.the_qa_company.qendpoint.core.options.HDTOptions; import com.the_qa_company.qendpoint.core.options.HDTOptionsKeys; -import com.the_qa_company.qendpoint.core.storage.TempBuffIn; -import com.the_qa_company.qendpoint.core.storage.TempBuffOut; +import it.unimi.dsi.fastutil.io.FastBufferedInputStream; +import it.unimi.dsi.fastutil.io.FastBufferedOutputStream; import com.the_qa_company.qendpoint.core.triples.TripleID; import com.the_qa_company.qendpoint.core.util.BitUtil; import com.the_qa_company.qendpoint.core.util.concurrent.KWayMerger; @@ -132,7 +132,7 @@ public static BitmapTriplesIndex map(Path file, FileChannel channel, BitmapTripl } } - CountInputStream stream = new CountInputStream(new TempBuffIn(Channels.newInputStream(channel))); + CountInputStream stream = new CountInputStream(new FastBufferedInputStream(Channels.newInputStream(channel))); stream.skipNBytes(headerSize); String orderCfg = IOUtil.readSizedString(stream, ProgressListener.ignore()); @@ -317,7 +317,7 @@ public static void generateIndex(BitmapTriples triples, Path destination, Triple seqZ.trimToSize(); // saving the index - try (OutputStream output = new TempBuffOut(Files.newOutputStream(destination))) { + try (OutputStream output = new FastBufferedOutputStream(Files.newOutputStream(destination))) { output.write(MAGIC); IOUtil.writeLong(output, signature(triples)); diff --git a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/util/LargeFakeDataSetStreamSupplier.java b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/util/LargeFakeDataSetStreamSupplier.java index 685601cb4..1a18b7f93 100644 --- a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/util/LargeFakeDataSetStreamSupplier.java +++ b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/util/LargeFakeDataSetStreamSupplier.java @@ -7,7 +7,7 @@ import com.the_qa_company.qendpoint.core.iterator.utils.MapIterator; import com.the_qa_company.qendpoint.core.options.HDTOptions; import com.the_qa_company.qendpoint.core.quad.QuadString; -import com.the_qa_company.qendpoint.core.storage.TempBuffOut; +import it.unimi.dsi.fastutil.io.FastBufferedOutputStream; import com.the_qa_company.qendpoint.core.triples.TripleString; import com.the_qa_company.qendpoint.core.util.concurrent.ExceptionThread; import com.the_qa_company.qendpoint.core.util.string.ByteStringUtil; @@ -195,7 +195,7 @@ public void createNTFile(Path file) throws IOException { */ public void createNTFile(Path file, CompressionType compressionType) throws IOException { try (Writer writer = new OutputStreamWriter( - new TempBuffOut(compressionType.compress(Files.newOutputStream(file))))) { + new FastBufferedOutputStream(compressionType.compress(Files.newOutputStream(file))))) { createNTFile(writer); } } diff --git a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/util/Profiler.java b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/util/Profiler.java index 70403011a..9139d824f 100644 --- a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/util/Profiler.java +++ b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/util/Profiler.java @@ -2,8 +2,8 @@ import com.the_qa_company.qendpoint.core.options.HDTOptions; import com.the_qa_company.qendpoint.core.options.HDTOptionsKeys; -import com.the_qa_company.qendpoint.core.storage.TempBuffIn; -import com.the_qa_company.qendpoint.core.storage.TempBuffOut; +import it.unimi.dsi.fastutil.io.FastBufferedInputStream; +import it.unimi.dsi.fastutil.io.FastBufferedOutputStream; import java.io.BufferedInputStream; import java.io.BufferedOutputStream; @@ -50,7 +50,7 @@ public static Profiler getProfilerById(long id) { */ public static Profiler readFromDisk(Path inputPath) throws IOException { Profiler p = new Profiler(""); - try (InputStream is = new TempBuffIn(Files.newInputStream(inputPath))) { + try (InputStream is = new FastBufferedInputStream(Files.newInputStream(inputPath))) { for (byte b : HEADER) { if (is.read() != b) { throw new IOException("Missing header for the profiling file!"); @@ -288,7 +288,7 @@ public void writeProfiling() throws IOException { * @param outputPath output path */ public void writeToDisk(Path outputPath) throws IOException { - try (OutputStream os = new TempBuffOut(Files.newOutputStream(outputPath))) { + try (OutputStream os = new FastBufferedOutputStream(Files.newOutputStream(outputPath))) { for (byte b : HEADER) { os.write(b); } diff --git a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/util/RDFInfo.java b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/util/RDFInfo.java index 9e510da55..728890f0e 100644 --- a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/util/RDFInfo.java +++ b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/util/RDFInfo.java @@ -18,7 +18,6 @@ package com.the_qa_company.qendpoint.core.util; -import java.io.BufferedInputStream; import java.io.FileInputStream; import java.io.IOException; import java.io.InputStream; @@ -27,7 +26,7 @@ import com.the_qa_company.qendpoint.core.exceptions.ParserException; import com.the_qa_company.qendpoint.core.options.HDTOptions; import com.the_qa_company.qendpoint.core.rdf.RDFParserCallback; -import com.the_qa_company.qendpoint.core.storage.TempBuffIn; +import it.unimi.dsi.fastutil.io.FastBufferedInputStream; /** * A class for getting basic information about a file @@ -137,7 +136,7 @@ public static float getCompression(HDTOptions specs) { */ public static long countLines(String filename, RDFParserCallback parser, RDFNotation notation) throws IOException, ParserException { - InputStream is = new TempBuffIn(new FileInputStream(filename)); + InputStream is = new FastBufferedInputStream(new FileInputStream(filename)); try { byte[] c = new byte[1024]; int count = 0; diff --git a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/util/io/CloseSuppressPath.java b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/util/io/CloseSuppressPath.java index c3e77ccd3..92aed90c8 100644 --- a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/util/io/CloseSuppressPath.java +++ b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/util/io/CloseSuppressPath.java @@ -1,7 +1,7 @@ package com.the_qa_company.qendpoint.core.util.io; -import com.the_qa_company.qendpoint.core.storage.TempBuffIn; -import com.the_qa_company.qendpoint.core.storage.TempBuffOut; +import it.unimi.dsi.fastutil.io.FastBufferedInputStream; +import it.unimi.dsi.fastutil.io.FastBufferedOutputStream; import java.io.BufferedInputStream; import java.io.BufferedOutputStream; @@ -235,7 +235,7 @@ public Spliterator spliterator() { } public InputStream openInputStream(int bufferSize, OpenOption... options) throws IOException { - return new TempBuffIn(openInputStream(options), bufferSize); + return new FastBufferedInputStream(openInputStream(options), bufferSize); } public InputStream openInputStream(OpenOption... options) throws IOException { @@ -247,7 +247,7 @@ private OutputStream openOutputStream(OpenOption... options) throws IOException } public OutputStream openOutputStream(int bufferSize, OpenOption... options) throws IOException { - return new TempBuffOut(openOutputStream(options), bufferSize); + return new FastBufferedOutputStream(openOutputStream(options), bufferSize); } /** diff --git a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/util/io/IOUtil.java b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/util/io/IOUtil.java index 019bf1cae..3d5ea468c 100644 --- a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/util/io/IOUtil.java +++ b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/util/io/IOUtil.java @@ -25,7 +25,7 @@ import com.the_qa_company.qendpoint.core.listener.ProgressListener; import com.the_qa_company.qendpoint.core.options.HDTOptions; import com.the_qa_company.qendpoint.core.options.HDTOptionsKeys; -import com.the_qa_company.qendpoint.core.storage.TempBuffIn; +import it.unimi.dsi.fastutil.io.FastBufferedInputStream; import com.the_qa_company.qendpoint.core.unsafe.MemoryUtils; import com.the_qa_company.qendpoint.core.unsafe.UnsafeLongArray; import com.the_qa_company.qendpoint.core.util.StringUtil; @@ -395,9 +395,9 @@ public static InputStream getFileInputStream(String fileName, boolean uncompress con.connect(); input = con.getInputStream(); } else if (name.equals("-")) { - input = new TempBuffIn(System.in); + input = new FastBufferedInputStream(System.in); } else { - input = new TempBuffIn(new FileInputStream(fileName)); + input = new FastBufferedInputStream(new FileInputStream(fileName)); } if (!skipHandled) { input.skipNBytes(startLen); @@ -616,7 +616,7 @@ public static void copy(InputStream is, OutputStream os, ProgressListener pl, lo } public static void decompressGzip(File src, File trgt) throws IOException { - try (InputStream in = new GZIPInputStream(new TempBuffIn(new FileInputStream(src)))) { + try (InputStream in = new GZIPInputStream(new FastBufferedInputStream(new FileInputStream(src)))) { Files.copy(in, trgt.toPath()); } } diff --git a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/util/nsd/NamespaceData.java b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/util/nsd/NamespaceData.java index d8d4674dc..f912ef239 100644 --- a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/util/nsd/NamespaceData.java +++ b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/util/nsd/NamespaceData.java @@ -3,8 +3,8 @@ import com.the_qa_company.qendpoint.core.compact.integer.VByte; import com.the_qa_company.qendpoint.core.listener.ProgressListener; import com.the_qa_company.qendpoint.core.storage.QEPCoreException; -import com.the_qa_company.qendpoint.core.storage.TempBuffIn; -import com.the_qa_company.qendpoint.core.storage.TempBuffOut; +import it.unimi.dsi.fastutil.io.FastBufferedInputStream; +import it.unimi.dsi.fastutil.io.FastBufferedOutputStream; import com.the_qa_company.qendpoint.core.util.crc.CRC32; import com.the_qa_company.qendpoint.core.util.crc.CRCOutputStream; import com.the_qa_company.qendpoint.core.util.io.IOUtil; @@ -142,7 +142,7 @@ public void sync() throws QEPCoreException { */ public void load() throws QEPCoreException { synchronized (syncObject) { - try (InputStream is = new TempBuffIn(Files.newInputStream(location))) { + try (InputStream is = new FastBufferedInputStream(Files.newInputStream(location))) { // check the binary magic byte[] header = is.readNBytes(MAGIC.length + 1); if (header.length < MAGIC.length + 1) { @@ -195,7 +195,7 @@ public void save(boolean onlyIfUpdated) throws QEPCoreException { return; // not updated } ProgressListener pl = ProgressListener.ignore(); - try (OutputStream osh = new TempBuffOut(Files.newOutputStream(location))) { + try (OutputStream osh = new FastBufferedOutputStream(Files.newOutputStream(location))) { // write magic and version osh.write(MAGIC); osh.write(NS_VERSION); diff --git a/qendpoint-core/src/test/java/com/the_qa_company/qendpoint/core/compact/bitmap/MultiRoaringBitmapTest.java b/qendpoint-core/src/test/java/com/the_qa_company/qendpoint/core/compact/bitmap/MultiRoaringBitmapTest.java index b736dda25..71cd2fb49 100644 --- a/qendpoint-core/src/test/java/com/the_qa_company/qendpoint/core/compact/bitmap/MultiRoaringBitmapTest.java +++ b/qendpoint-core/src/test/java/com/the_qa_company/qendpoint/core/compact/bitmap/MultiRoaringBitmapTest.java @@ -1,8 +1,8 @@ package com.the_qa_company.qendpoint.core.compact.bitmap; import com.the_qa_company.qendpoint.core.listener.ProgressListener; -import com.the_qa_company.qendpoint.core.storage.TempBuffIn; -import com.the_qa_company.qendpoint.core.storage.TempBuffOut; +import it.unimi.dsi.fastutil.io.FastBufferedInputStream; +import it.unimi.dsi.fastutil.io.FastBufferedOutputStream; import com.the_qa_company.qendpoint.core.util.io.AbstractMapMemoryTest; import org.apache.commons.io.file.PathUtils; import org.junit.After; @@ -56,7 +56,7 @@ public void serialSyncTest() throws IOException { map.set(0, 90, true); } - try (InputStream stream = new TempBuffIn(Files.newInputStream(output)); + try (InputStream stream = new FastBufferedInputStream(Files.newInputStream(output)); MultiRoaringBitmap map = MultiRoaringBitmap.load(stream)) { for (int i = 0; i < 100; i++) { switch (i) { @@ -106,7 +106,7 @@ public void largeSerialSyncTest() throws IOException { assertTrue(map.access(layer, position)); } - try (OutputStream out = new TempBuffOut(Files.newOutputStream(output))) { + try (OutputStream out = new FastBufferedOutputStream(Files.newOutputStream(output))) { map.save(out, ProgressListener.ignore()); } } diff --git a/qendpoint-core/src/test/java/com/the_qa_company/qendpoint/core/dictionary/impl/kcat/KCatMergerTest.java b/qendpoint-core/src/test/java/com/the_qa_company/qendpoint/core/dictionary/impl/kcat/KCatMergerTest.java index 77996f0b6..b21cb0ae9 100644 --- a/qendpoint-core/src/test/java/com/the_qa_company/qendpoint/core/dictionary/impl/kcat/KCatMergerTest.java +++ b/qendpoint-core/src/test/java/com/the_qa_company/qendpoint/core/dictionary/impl/kcat/KCatMergerTest.java @@ -20,8 +20,8 @@ import com.the_qa_company.qendpoint.core.listener.ProgressListener; import com.the_qa_company.qendpoint.core.options.HDTOptions; import com.the_qa_company.qendpoint.core.options.HDTOptionsKeys; -import com.the_qa_company.qendpoint.core.storage.TempBuffIn; -import com.the_qa_company.qendpoint.core.storage.TempBuffOut; +import it.unimi.dsi.fastutil.io.FastBufferedInputStream; +import it.unimi.dsi.fastutil.io.FastBufferedOutputStream; import com.the_qa_company.qendpoint.core.triples.IteratorTripleString; import com.the_qa_company.qendpoint.core.triples.TripleString; import com.the_qa_company.qendpoint.core.util.LargeFakeDataSetStreamSupplier; @@ -149,7 +149,7 @@ public void mergerTest() throws ParserException, IOException, InterruptedExcepti merger.startMerger(); // create DictionaryPrivate dict = merger.buildDictionary(); - try (OutputStream stream = new TempBuffOut(Files.newOutputStream(dictFile))) { + try (OutputStream stream = new FastBufferedOutputStream(Files.newOutputStream(dictFile))) { writeSection(dict.getShared(), stream); writeSection(dict.getSubjects(), stream); writeSection(dict.getPredicates(), stream); @@ -240,7 +240,7 @@ public void mergerTest() throws ParserException, IOException, InterruptedExcepti } } } - try (InputStream stream = new TempBuffIn(Files.newInputStream(dictFile))) { + try (InputStream stream = new FastBufferedInputStream(Files.newInputStream(dictFile))) { // read the sections try (DictionarySection sh = loadSection(stream); DictionarySection su = loadSection(stream); @@ -483,7 +483,7 @@ public void diffMergerTest() throws ParserException, IOException, InterruptedExc merger.startMerger(); // create DictionaryPrivate dict = merger.buildDictionary(); - try (OutputStream stream = new TempBuffOut(Files.newOutputStream(dictFile))) { + try (OutputStream stream = new FastBufferedOutputStream(Files.newOutputStream(dictFile))) { writeSection(dict.getShared(), stream); writeSection(dict.getSubjects(), stream); writeSection(dict.getPredicates(), stream); @@ -501,7 +501,7 @@ public void diffMergerTest() throws ParserException, IOException, InterruptedExc } } } - try (InputStream stream = new TempBuffIn(Files.newInputStream(dictFile))) { + try (InputStream stream = new FastBufferedInputStream(Files.newInputStream(dictFile))) { // read the sections try (DictionarySection sh = loadSection(stream); DictionarySection su = loadSection(stream); diff --git a/qendpoint-core/src/test/java/com/the_qa_company/qendpoint/core/dictionary/impl/section/WriteDictionarySectionTest.java b/qendpoint-core/src/test/java/com/the_qa_company/qendpoint/core/dictionary/impl/section/WriteDictionarySectionTest.java index 187179aaf..55cd7ac06 100644 --- a/qendpoint-core/src/test/java/com/the_qa_company/qendpoint/core/dictionary/impl/section/WriteDictionarySectionTest.java +++ b/qendpoint-core/src/test/java/com/the_qa_company/qendpoint/core/dictionary/impl/section/WriteDictionarySectionTest.java @@ -3,7 +3,7 @@ import com.the_qa_company.qendpoint.core.iterator.utils.MapIterator; import com.the_qa_company.qendpoint.core.listener.ProgressListener; import com.the_qa_company.qendpoint.core.options.HDTOptions; -import com.the_qa_company.qendpoint.core.storage.TempBuffOut; +import it.unimi.dsi.fastutil.io.FastBufferedOutputStream; import com.the_qa_company.qendpoint.core.util.LargeFakeDataSetStreamSupplier; import com.the_qa_company.qendpoint.core.util.string.ByteString; import org.apache.commons.io.file.PathUtils; @@ -50,10 +50,10 @@ public void appenderTest() throws IOException { } Path t1Save = dir.resolve("t1.save"); Path t2Save = dir.resolve("t2.save"); - try (OutputStream os = new TempBuffOut(Files.newOutputStream(t1Save))) { + try (OutputStream os = new FastBufferedOutputStream(Files.newOutputStream(t1Save))) { section1.save(os, ProgressListener.ignore()); } - try (OutputStream os = new TempBuffOut(Files.newOutputStream(t2Save))) { + try (OutputStream os = new FastBufferedOutputStream(Files.newOutputStream(t2Save))) { section2.save(os, ProgressListener.ignore()); } diff --git a/qendpoint-core/src/test/java/com/the_qa_company/qendpoint/core/rdf/parsers/TarTest.java b/qendpoint-core/src/test/java/com/the_qa_company/qendpoint/core/rdf/parsers/TarTest.java index a54b3a876..2257ed0ad 100644 --- a/qendpoint-core/src/test/java/com/the_qa_company/qendpoint/core/rdf/parsers/TarTest.java +++ b/qendpoint-core/src/test/java/com/the_qa_company/qendpoint/core/rdf/parsers/TarTest.java @@ -5,7 +5,7 @@ import java.io.InputStream; import java.util.zip.GZIPInputStream; -import com.the_qa_company.qendpoint.core.storage.TempBuffIn; +import it.unimi.dsi.fastutil.io.FastBufferedInputStream; import org.apache.commons.compress.archivers.tar.TarArchiveEntry; import org.apache.commons.compress.archivers.tar.TarArchiveInputStream; import com.the_qa_company.qendpoint.core.util.io.CountInputStream; @@ -15,9 +15,9 @@ public class TarTest { public static void main(String[] args) throws Throwable { - InputStream input = new CountInputStream( - new TempBuffIn(new GZIPInputStream(new FileInputStream("/Users/mck/rdf/dataset/tgztest.tar.gz")))); -// InputStream input = new CountInputStream(new TempBuffIn(new FileInputStream("/Users/mck/rdf/dataset/tgztest.tar"))); + InputStream input = new CountInputStream(new FastBufferedInputStream( + new GZIPInputStream(new FileInputStream("/Users/mck/rdf/dataset/tgztest.tar.gz")))); +// InputStream input = new CountInputStream(new FastBufferedInputStream(new FileInputStream("/Users/mck/rdf/dataset/tgztest.tar"))); final TarArchiveInputStream debInputStream = new TarArchiveInputStream(input); TarArchiveEntry entry; diff --git a/qendpoint-core/src/test/java/com/the_qa_company/qendpoint/core/triples/impl/BitmapQuadTriplesTest.java b/qendpoint-core/src/test/java/com/the_qa_company/qendpoint/core/triples/impl/BitmapQuadTriplesTest.java index 1cd4a7714..21934a22e 100644 --- a/qendpoint-core/src/test/java/com/the_qa_company/qendpoint/core/triples/impl/BitmapQuadTriplesTest.java +++ b/qendpoint-core/src/test/java/com/the_qa_company/qendpoint/core/triples/impl/BitmapQuadTriplesTest.java @@ -4,8 +4,8 @@ import com.the_qa_company.qendpoint.core.enums.TripleComponentOrder; import com.the_qa_company.qendpoint.core.listener.ProgressListener; import com.the_qa_company.qendpoint.core.options.ControlInformation; -import com.the_qa_company.qendpoint.core.storage.TempBuffIn; -import com.the_qa_company.qendpoint.core.storage.TempBuffOut; +import it.unimi.dsi.fastutil.io.FastBufferedInputStream; +import it.unimi.dsi.fastutil.io.FastBufferedOutputStream; import com.the_qa_company.qendpoint.core.triples.IteratorTripleID; import com.the_qa_company.qendpoint.core.triples.TripleID; import com.the_qa_company.qendpoint.core.util.io.AbstractMapMemoryTest; @@ -147,14 +147,14 @@ public void triplesTest() throws IOException { assertEquals(it2.next(), it.next()); } - try (OutputStream stream = new TempBuffOut(Files.newOutputStream(path))) { + try (OutputStream stream = new FastBufferedOutputStream(Files.newOutputStream(path))) { triples.save(stream, new ControlInformation(), ProgressListener.ignore()); } } // load try (BitmapQuadTriples triples = new BitmapQuadTriples()) { - try (InputStream stream = new TempBuffIn(Files.newInputStream(path))) { + try (InputStream stream = new FastBufferedInputStream(Files.newInputStream(path))) { ControlInformation ci = new ControlInformation(); ci.load(stream); triples.load(stream, ci, ProgressListener.ignore()); @@ -170,7 +170,7 @@ public void triplesTest() throws IOException { // map try (BitmapQuadTriples triples = new BitmapQuadTriples()) { - try (InputStream stream = new TempBuffIn(Files.newInputStream(path))) { + try (InputStream stream = new FastBufferedInputStream(Files.newInputStream(path))) { CountInputStream cstream = new CountInputStream(stream); triples.mapFromFile(cstream, path.toFile(), ProgressListener.ignore()); } diff --git a/qendpoint-core/src/test/java/com/the_qa_company/qendpoint/core/util/LargeFakeDataSetStreamSupplierTest.java b/qendpoint-core/src/test/java/com/the_qa_company/qendpoint/core/util/LargeFakeDataSetStreamSupplierTest.java index 30b09b390..c52fc7af6 100644 --- a/qendpoint-core/src/test/java/com/the_qa_company/qendpoint/core/util/LargeFakeDataSetStreamSupplierTest.java +++ b/qendpoint-core/src/test/java/com/the_qa_company/qendpoint/core/util/LargeFakeDataSetStreamSupplierTest.java @@ -10,7 +10,7 @@ import com.the_qa_company.qendpoint.core.options.HDTSpecification; import com.the_qa_company.qendpoint.core.rdf.RDFParserCallback; import com.the_qa_company.qendpoint.core.rdf.RDFParserFactory; -import com.the_qa_company.qendpoint.core.storage.TempBuffIn; +import it.unimi.dsi.fastutil.io.FastBufferedInputStream; import com.the_qa_company.qendpoint.core.triples.TripleString; import com.the_qa_company.qendpoint.core.triples.impl.utils.HDTTestUtils; import org.junit.Rule; @@ -185,8 +185,8 @@ public void mergeTest() throws IOException, ParserException, NotFoundException { RDFParserCallback parser = RDFParserFactory.getParserCallback(RDFNotation.NTRIPLES, HDTOptions.of(Map.of(HDTOptionsKeys.NT_SIMPLE_PARSER_KEY, "true"))); try { - try (InputStream stream = new TempBuffIn(Files.newInputStream(p12)); - InputStream stream2 = new TempBuffIn(Files.newInputStream(p3)); + try (InputStream stream = new FastBufferedInputStream(Files.newInputStream(p12)); + InputStream stream2 = new FastBufferedInputStream(Files.newInputStream(p3)); PipedCopyIterator it1 = RDFParserFactory.readAsIterator(parser, stream, "http://w", true, RDFNotation.NTRIPLES); PipedCopyIterator it2 = RDFParserFactory.readAsIterator(parser, stream2, "http://w", diff --git a/qendpoint-store/src/main/java/com/the_qa_company/qendpoint/compiler/SailCompiler.java b/qendpoint-store/src/main/java/com/the_qa_company/qendpoint/compiler/SailCompiler.java index 85289a629..498c15311 100644 --- a/qendpoint-store/src/main/java/com/the_qa_company/qendpoint/compiler/SailCompiler.java +++ b/qendpoint-store/src/main/java/com/the_qa_company/qendpoint/compiler/SailCompiler.java @@ -8,7 +8,7 @@ import com.the_qa_company.qendpoint.compiler.source.EmptyTripleSourceGetter; import com.the_qa_company.qendpoint.compiler.source.ModelTripleSourceGetter; import com.the_qa_company.qendpoint.compiler.source.SailTripleSourceModel; -import com.the_qa_company.qendpoint.core.storage.TempBuffIn; +import it.unimi.dsi.fastutil.io.FastBufferedInputStream; import com.the_qa_company.qendpoint.utils.sail.linked.LinkedSail; import org.eclipse.rdf4j.common.iteration.CloseableIteration; import org.eclipse.rdf4j.model.IRI; @@ -134,7 +134,7 @@ public void load(Path rdfFile) throws IOException { RDFFormat format = Rio.getParserFormatForFileName(rdfFile.getFileName().toString()) .orElseThrow(() -> new IllegalArgumentException("Can't find parser for file: " + rdfFile)); - try (InputStream stream = new TempBuffIn(Files.newInputStream(rdfFile))) { + try (InputStream stream = new FastBufferedInputStream(Files.newInputStream(rdfFile))) { load(stream, format); } } diff --git a/qendpoint-store/src/main/java/com/the_qa_company/qendpoint/store/EndpointStore.java b/qendpoint-store/src/main/java/com/the_qa_company/qendpoint/store/EndpointStore.java index c06882ca9..c7e4f8924 100644 --- a/qendpoint-store/src/main/java/com/the_qa_company/qendpoint/store/EndpointStore.java +++ b/qendpoint-store/src/main/java/com/the_qa_company/qendpoint/store/EndpointStore.java @@ -11,7 +11,6 @@ import com.the_qa_company.qendpoint.core.hdt.HDTManager; import com.the_qa_company.qendpoint.core.options.HDTOptions; import com.the_qa_company.qendpoint.core.options.HDTOptionsKeys; -import com.the_qa_company.qendpoint.core.storage.TempBuffOut; import com.the_qa_company.qendpoint.core.triples.IteratorTripleID; import com.the_qa_company.qendpoint.core.triples.TripleID; import com.the_qa_company.qendpoint.core.triples.TripleString; @@ -20,6 +19,7 @@ import com.the_qa_company.qendpoint.model.HDTValue; import com.the_qa_company.qendpoint.utils.BitArrayDisk; import com.the_qa_company.qendpoint.utils.CloseSafeHDT; +import it.unimi.dsi.fastutil.io.FastBufferedOutputStream; import org.apache.commons.io.FileUtils; import org.apache.commons.io.file.PathUtils; import org.eclipse.rdf4j.common.concurrent.locks.Lock; @@ -701,7 +701,8 @@ public void initTempDump(boolean isRestarting) { if (!file.exists()) { Files.createFile(file.toPath()); } - OutputStream rdfWriterTempTriplesOut = new TempBuffOut(new FileOutputStream(file, isRestarting)); + OutputStream rdfWriterTempTriplesOut = new FastBufferedOutputStream( + new FileOutputStream(file, isRestarting)); this.rdfWriterTempTriples = graph ? new NQuadsWriter(rdfWriterTempTriplesOut) : new NTriplesWriter(rdfWriterTempTriplesOut); this.rdfWriterTempTriples.startRDF(); diff --git a/qendpoint-store/src/main/java/com/the_qa_company/qendpoint/store/MergeRunnable.java b/qendpoint-store/src/main/java/com/the_qa_company/qendpoint/store/MergeRunnable.java index 639886317..d45ded52b 100644 --- a/qendpoint-store/src/main/java/com/the_qa_company/qendpoint/store/MergeRunnable.java +++ b/qendpoint-store/src/main/java/com/the_qa_company/qendpoint/store/MergeRunnable.java @@ -4,7 +4,7 @@ import com.the_qa_company.qendpoint.core.compact.bitmap.MultiLayerBitmapWrapper; import com.the_qa_company.qendpoint.core.dictionary.Dictionary; import com.the_qa_company.qendpoint.core.enums.TripleComponentOrder; -import com.the_qa_company.qendpoint.core.storage.TempBuffOut; +import it.unimi.dsi.fastutil.io.FastBufferedOutputStream; import com.the_qa_company.qendpoint.store.exception.EndpointStoreException; import com.the_qa_company.qendpoint.utils.BitArrayDisk; import com.the_qa_company.qendpoint.utils.OverrideHDTOptions; @@ -926,7 +926,7 @@ private void createHDTDump(String rdfInput, String hdtOutput) throws IOException } private void writeTempFile(RepositoryConnection connection, String file, boolean graph) throws IOException { - try (OutputStream out = new TempBuffOut(new FileOutputStream(file))) { + try (OutputStream out = new FastBufferedOutputStream(new FileOutputStream(file))) { RDFWriter writer = Rio.createWriter( endpoint.getHdt().getDictionary().supportGraphs() ? RDFFormat.NQUADS : RDFFormat.NTRIPLES, out); RepositoryResult repositoryResult = connection.getStatements(null, null, null, false); diff --git a/qendpoint-store/src/main/java/com/the_qa_company/qendpoint/tools/QEPSearch.java b/qendpoint-store/src/main/java/com/the_qa_company/qendpoint/tools/QEPSearch.java index fe06f33b2..737375b8f 100644 --- a/qendpoint-store/src/main/java/com/the_qa_company/qendpoint/tools/QEPSearch.java +++ b/qendpoint-store/src/main/java/com/the_qa_company/qendpoint/tools/QEPSearch.java @@ -17,7 +17,7 @@ import com.the_qa_company.qendpoint.core.options.HDTOptions; import com.the_qa_company.qendpoint.core.options.HDTOptionsKeys; import com.the_qa_company.qendpoint.core.rdf.parsers.RDFDeltaFileParser; -import com.the_qa_company.qendpoint.core.storage.TempBuffIn; +import it.unimi.dsi.fastutil.io.FastBufferedInputStream; import com.the_qa_company.qendpoint.core.tools.HDTVerify; import com.the_qa_company.qendpoint.core.triples.IteratorTripleString; import com.the_qa_company.qendpoint.core.triples.TripleString; @@ -1029,7 +1029,7 @@ private void executeDeltaFile() throws IOException { HDTOptionsKeys.PARSER_DELTAFILE_NO_EXCEPTION, true); try (RDFDeltaFileParser.DeltaFileReader reader = new RDFDeltaFileParser.DeltaFileReader( - new TempBuffIn(Files.newInputStream(file)), spec)) { + new FastBufferedInputStream(Files.newInputStream(file)), spec)) { console.printLine(console.color(5, 5, 1) + "files .. " + console.colorReset() + reader.getSize()); console.printLine(console.color(5, 5, 1) + "start .. " + console.colorReset() + reader.getStart()); diff --git a/qendpoint-store/src/main/java/com/the_qa_company/qendpoint/utils/BitArrayDisk.java b/qendpoint-store/src/main/java/com/the_qa_company/qendpoint/utils/BitArrayDisk.java index c29271cd9..f79e432a1 100644 --- a/qendpoint-store/src/main/java/com/the_qa_company/qendpoint/utils/BitArrayDisk.java +++ b/qendpoint-store/src/main/java/com/the_qa_company/qendpoint/utils/BitArrayDisk.java @@ -1,6 +1,6 @@ package com.the_qa_company.qendpoint.utils; -import com.the_qa_company.qendpoint.core.storage.TempBuffIn; +import it.unimi.dsi.fastutil.io.FastBufferedInputStream; import com.the_qa_company.qendpoint.store.exception.EndpointStoreException; import org.eclipse.rdf4j.common.io.NioFile; import com.the_qa_company.qendpoint.core.compact.bitmap.ModifiableBitmap; @@ -141,7 +141,8 @@ private void initWordsArray(long nbits) throws IOException { int lastNonZero = -1; // read previous values - try (InputStream is = new TempBuffIn(Files.newInputStream(this.output.getFile().toPath()))) { + try (InputStream is = new FastBufferedInputStream( + Files.newInputStream(this.output.getFile().toPath()))) { // skip header is.skipNBytes(8); for (int i = 0; i < this.words.length; i++) { diff --git a/qendpoint-store/src/main/java/com/the_qa_company/qendpoint/utils/FormatUtils.java b/qendpoint-store/src/main/java/com/the_qa_company/qendpoint/utils/FormatUtils.java index 038064cb6..9102fee25 100644 --- a/qendpoint-store/src/main/java/com/the_qa_company/qendpoint/utils/FormatUtils.java +++ b/qendpoint-store/src/main/java/com/the_qa_company/qendpoint/utils/FormatUtils.java @@ -2,7 +2,7 @@ import com.the_qa_company.qendpoint.core.options.ControlInfo; import com.the_qa_company.qendpoint.core.options.ControlInformation; -import com.the_qa_company.qendpoint.core.storage.TempBuffIn; +import it.unimi.dsi.fastutil.io.FastBufferedInputStream; import org.eclipse.rdf4j.query.resultio.QueryResultFormat; import org.eclipse.rdf4j.query.resultio.TupleQueryResultWriterRegistry; import org.eclipse.rdf4j.rio.RDFFormat; @@ -88,7 +88,7 @@ public static Optional getRDFWriterFormat(String acceptHeader) { */ public static byte[] readCookie(Path file, int size) { try (InputStream stream = Files.newInputStream(file)) { - return (size > 0x1000 ? new TempBuffIn(stream) : stream).readNBytes(size); + return (size > 0x1000 ? new FastBufferedInputStream(stream) : stream).readNBytes(size); } catch (IOException e) { return new byte[0]; } diff --git a/qendpoint-store/src/test/java/com/the_qa_company/qendpoint/store/MergeRestartTest.java b/qendpoint-store/src/test/java/com/the_qa_company/qendpoint/store/MergeRestartTest.java index 69c86dfd4..c4f69590d 100644 --- a/qendpoint-store/src/test/java/com/the_qa_company/qendpoint/store/MergeRestartTest.java +++ b/qendpoint-store/src/test/java/com/the_qa_company/qendpoint/store/MergeRestartTest.java @@ -2,7 +2,7 @@ import com.the_qa_company.qendpoint.core.compact.bitmap.MultiLayerBitmapWrapper; import com.the_qa_company.qendpoint.core.enums.TripleComponentOrder; -import com.the_qa_company.qendpoint.core.storage.TempBuffOut; +import it.unimi.dsi.fastutil.io.FastBufferedOutputStream; import com.the_qa_company.qendpoint.utils.BitArrayDisk; import org.apache.commons.io.FileUtils; import org.eclipse.rdf4j.common.iteration.CloseableIteration; @@ -889,7 +889,8 @@ private void executeTestRemoveHDT(File out, SailRepository repo, int id, int cou connection.remove(stm); }); writeInfoCount(out, count); - try (OutputStream buff = new TempBuffOut(new FileOutputStream(out.getAbsolutePath() + ".delta", true))) { + try (OutputStream buff = new FastBufferedOutputStream( + new FileOutputStream(out.getAbsolutePath() + ".delta", true))) { buff.write(("REM HDT " + id + " / " + count + "\n").getBytes(StandardCharsets.UTF_8)); } @@ -913,7 +914,8 @@ private void executeTestRemoveRDF(File out, SailRepository repo, int id, int cou connection.remove(stm); }); writeInfoCount(out, count); - try (OutputStream buff = new TempBuffOut(new FileOutputStream(out.getAbsolutePath() + ".delta", true))) { + try (OutputStream buff = new FastBufferedOutputStream( + new FileOutputStream(out.getAbsolutePath() + ".delta", true))) { buff.write(("REM RDF " + id + " / " + count + "\n").getBytes(StandardCharsets.UTF_8)); } } @@ -936,7 +938,8 @@ private void executeTestAddRDF(File out, SailRepository repo, int id, int count) connection.add(stm); }); writeInfoCount(out, count); - try (OutputStream buff = new TempBuffOut(new FileOutputStream(out.getAbsolutePath() + ".delta", true))) { + try (OutputStream buff = new FastBufferedOutputStream( + new FileOutputStream(out.getAbsolutePath() + ".delta", true))) { buff.write(("ADD RDF " + id + " / " + count + "\n").getBytes(StandardCharsets.UTF_8)); } } @@ -960,7 +963,8 @@ private void executeTestAddHDT(File out, SailRepository repo, int id, int count) connection.add(stm); }); writeInfoCount(out, count); - try (OutputStream buff = new TempBuffOut(new FileOutputStream(out.getAbsolutePath() + ".delta", true))) { + try (OutputStream buff = new FastBufferedOutputStream( + new FileOutputStream(out.getAbsolutePath() + ".delta", true))) { buff.write(("ADD HDT " + id + " / " + count + "\n").getBytes(StandardCharsets.UTF_8)); } } diff --git a/qendpoint-store/src/test/java/com/the_qa_company/qendpoint/store/Utility.java b/qendpoint-store/src/test/java/com/the_qa_company/qendpoint/store/Utility.java index 632138faa..be44a033c 100644 --- a/qendpoint-store/src/test/java/com/the_qa_company/qendpoint/store/Utility.java +++ b/qendpoint-store/src/test/java/com/the_qa_company/qendpoint/store/Utility.java @@ -1,6 +1,6 @@ package com.the_qa_company.qendpoint.store; -import com.the_qa_company.qendpoint.core.storage.TempBuffOut; +import it.unimi.dsi.fastutil.io.FastBufferedOutputStream; import org.eclipse.rdf4j.model.IRI; import org.eclipse.rdf4j.model.Resource; import org.eclipse.rdf4j.model.Statement; @@ -161,7 +161,7 @@ public static Statement getFakeStatement(ValueFactory vf, int id) { private static void writeBigIndex(File file) throws IOException { ValueFactory vf = new MemValueFactory(); - try (OutputStream out = new TempBuffOut(new FileOutputStream(file))) { + try (OutputStream out = new FastBufferedOutputStream(new FileOutputStream(file))) { RDFWriter writer = Rio.createWriter(RDFFormat.NTRIPLES, out); writer.startRDF(); for (int i = 1; i <= COUNT; i++) { From d7fe3a7ddef10407f3c953f0aa1b631d12f1d253 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ha=CC=8Avard=20Ottestad?= Date: Wed, 19 Mar 2025 15:43:02 +0100 Subject: [PATCH 22/28] wip --- .../qendpoint/controller/Sparql.java | 4 +- qendpoint-core/pom.xml | 12 +++ .../compact/bitmap/MultiRoaringBitmap.java | 12 +-- .../compact/sequence/SequenceLog64Map.java | 7 +- .../impl/section/PFCDictionarySectionBig.java | 10 +- .../impl/section/PFCDictionarySectionMap.java | 5 +- .../qendpoint/core/hdt/HDTManagerImpl.java | 5 +- .../qendpoint/core/hdt/impl/HDTImpl.java | 30 +++--- .../qendpoint/core/hdt/impl/WriteHDTImpl.java | 5 +- .../impl/diskimport/SectionCompressor.java | 3 +- .../core/hdt/writer/TripleWriterHDT.java | 11 +-- .../core/hdt/writer/TripleWriterNtriples.java | 5 +- .../iterator/utils/AsyncIteratorFetcher.java | 15 ++- .../utils/AsyncIteratorFetcherUnordered.java | 17 ++++ .../core/iterator/utils/ParallelMerge.java | 2 +- .../utils/PipedCopyIteratorUnordered.java | 92 +++++++++++++++++-- .../core/iterator/utils/SizeFetcher.java | 5 +- .../qendpoint/core/options/ControlInfo.java | 5 +- .../core/options/HDTOptionsFile.java | 10 +- .../core/rdf/parsers/RDFParserRIOT.java | 11 +++ .../core/storage/QEPMapIdSorter.java | 17 ++-- .../core/triples/impl/BitmapTriplesCat.java | 5 +- .../triples/impl/BitmapTriplesIndexFile.java | 10 +- .../util/LargeFakeDataSetStreamSupplier.java | 5 +- .../qendpoint/core/util/Profiler.java | 10 +- .../qendpoint/core/util/RDFInfo.java | 4 +- .../core/util/io/CloseSuppressPath.java | 10 +- .../qendpoint/core/util/io/IOUtil.java | 9 +- .../util/io/compress/CompressNodeWriter.java | 1 + .../core/util/nsd/NamespaceData.java | 10 +- .../bitmap/MultiRoaringBitmapTest.java | 10 +- .../dictionary/impl/kcat/KCatMergerTest.java | 14 ++- .../section/WriteDictionarySectionTest.java | 7 +- .../core/hdt/impl/HasmacHDTImporterTest.java | 2 - .../parsers/AbstractNTriplesParserTest.java | 8 +- .../qendpoint/core/rdf/parsers/TarTest.java | 7 +- .../triples/impl/BitmapQuadTriplesTest.java | 12 +-- .../core/triples/impl/utils/HDTTestUtils.java | 1 - .../LargeFakeDataSetStreamSupplierTest.java | 7 +- .../qendpoint/compiler/SailCompiler.java | 5 +- .../qendpoint/store/EndpointStore.java | 6 +- .../qendpoint/store/MergeRunnable.java | 5 +- .../qendpoint/tools/QEPSearch.java | 5 +- .../qendpoint/utils/BitArrayDisk.java | 6 +- .../qendpoint/utils/FormatUtils.java | 5 +- .../qendpoint/store/MergeRestartTest.java | 11 +-- .../qendpoint/store/Utility.java | 5 +- 47 files changed, 273 insertions(+), 190 deletions(-) diff --git a/qendpoint-backend/src/main/java/com/the_qa_company/qendpoint/controller/Sparql.java b/qendpoint-backend/src/main/java/com/the_qa_company/qendpoint/controller/Sparql.java index 7b25ce0ca..629b80440 100644 --- a/qendpoint-backend/src/main/java/com/the_qa_company/qendpoint/controller/Sparql.java +++ b/qendpoint-backend/src/main/java/com/the_qa_company/qendpoint/controller/Sparql.java @@ -34,8 +34,8 @@ import org.springframework.stereotype.Component; import org.springframework.web.server.ServerWebInputException; -import javax.annotation.PostConstruct; -import javax.annotation.PreDestroy; +import jakarta.annotation.PostConstruct; +import jakarta.annotation.PreDestroy; import java.io.File; import java.io.FileOutputStream; import java.io.IOException; diff --git a/qendpoint-core/pom.xml b/qendpoint-core/pom.xml index f025669b5..f3d7ba238 100644 --- a/qendpoint-core/pom.xml +++ b/qendpoint-core/pom.xml @@ -128,5 +128,17 @@ fastutil 8.5.15 + + org.spf4j + spf4j-core + 8.10.0 + + + org.apache.avro + avro + + + + diff --git a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/compact/bitmap/MultiRoaringBitmap.java b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/compact/bitmap/MultiRoaringBitmap.java index a79adb7eb..7f302abca 100644 --- a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/compact/bitmap/MultiRoaringBitmap.java +++ b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/compact/bitmap/MultiRoaringBitmap.java @@ -3,15 +3,13 @@ import com.the_qa_company.qendpoint.core.exceptions.NotImplementedException; import com.the_qa_company.qendpoint.core.hdt.HDTVocabulary; import com.the_qa_company.qendpoint.core.listener.ProgressListener; -import it.unimi.dsi.fastutil.io.FastBufferedInputStream; -import it.unimi.dsi.fastutil.io.FastBufferedOutputStream; +import org.spf4j.io.BufferedInputStream; +import org.spf4j.io.BufferedOutputStream; import com.the_qa_company.qendpoint.core.util.io.CloseMappedByteBuffer; import com.the_qa_company.qendpoint.core.util.io.Closer; import com.the_qa_company.qendpoint.core.util.io.IOUtil; import org.roaringbitmap.RoaringBitmap; -import java.io.BufferedInputStream; -import java.io.BufferedOutputStream; import java.io.Closeable; import java.io.DataInputStream; import java.io.DataOutputStream; @@ -69,7 +67,7 @@ public static MultiRoaringBitmap load(InputStream input) throws IOException { * @throws IOException io exception when loading */ public static MultiRoaringBitmap load(Path input) throws IOException { - try (InputStream stream = new FastBufferedInputStream(Files.newInputStream(input))) { + try (InputStream stream = new BufferedInputStream(Files.newInputStream(input))) { return load(stream); } } @@ -341,7 +339,7 @@ private void closeStreamBitmap(int layer, int index) throws IOException { int sizeInBytes = handle.serializedSizeInBytes(); outputMax += sizeInBytes + 8 + 8 + 1; - OutputStream os = new FastBufferedOutputStream(Channels.newOutputStream(output.position(loc))); + OutputStream os = new BufferedOutputStream(Channels.newOutputStream(output.position(loc))); os.write(BLOCK_BITMAP); IOUtil.writeLong(os, sizeInBytes); IOUtil.writeLong(os, layer); @@ -356,7 +354,7 @@ private void closeStreamBitmap(int layer, int index) throws IOException { } public void save(Path output) throws IOException { - try (OutputStream stream = new FastBufferedOutputStream(Files.newOutputStream(output))) { + try (OutputStream stream = new BufferedOutputStream(Files.newOutputStream(output))) { save(stream); } } diff --git a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/compact/sequence/SequenceLog64Map.java b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/compact/sequence/SequenceLog64Map.java index f13252523..60bda9327 100644 --- a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/compact/sequence/SequenceLog64Map.java +++ b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/compact/sequence/SequenceLog64Map.java @@ -25,7 +25,7 @@ import com.the_qa_company.qendpoint.core.exceptions.NotImplementedException; import com.the_qa_company.qendpoint.core.hdt.HDTVocabulary; import com.the_qa_company.qendpoint.core.listener.ProgressListener; -import it.unimi.dsi.fastutil.io.FastBufferedInputStream; +import org.spf4j.io.BufferedInputStream; import com.the_qa_company.qendpoint.core.util.BitUtil; import com.the_qa_company.qendpoint.core.util.crc.CRC32; import com.the_qa_company.qendpoint.core.util.crc.CRC8; @@ -36,7 +36,6 @@ import com.the_qa_company.qendpoint.core.util.io.CountInputStream; import com.the_qa_company.qendpoint.core.util.io.IOUtil; -import java.io.BufferedInputStream; import java.io.Closeable; import java.io.File; import java.io.FileInputStream; @@ -66,7 +65,7 @@ public class SequenceLog64Map implements Sequence, Closeable { public SequenceLog64Map(File f) throws IOException { // Read from the beginning of the file - this(new CountInputStream(new FastBufferedInputStream(new FileInputStream(f))), f, true); + this(new CountInputStream(new BufferedInputStream(new FileInputStream(f))), f, true); } public SequenceLog64Map(CountInputStream in, File f) throws IOException { @@ -163,7 +162,7 @@ private void mapFiles(File f, long base) throws IOException { // FIXME: Bug in the previous code, find what because it should be more // efficient - CountInputStream in = new CountInputStream(new FastBufferedInputStream(new FileInputStream(f))); + CountInputStream in = new CountInputStream(new BufferedInputStream(new FileInputStream(f))); IOUtil.skip(in, base + ((numwords - 1) * 8L)); // System.out.println("Last word starts at: "+in.getTotalBytes()); // Read only used bits from last entry (byte aligned, little endian) diff --git a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/dictionary/impl/section/PFCDictionarySectionBig.java b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/dictionary/impl/section/PFCDictionarySectionBig.java index 6ed1f0892..0c74a2363 100644 --- a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/dictionary/impl/section/PFCDictionarySectionBig.java +++ b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/dictionary/impl/section/PFCDictionarySectionBig.java @@ -19,8 +19,6 @@ package com.the_qa_company.qendpoint.core.dictionary.impl.section; -import java.io.BufferedInputStream; -import java.io.BufferedOutputStream; import java.io.ByteArrayOutputStream; import java.io.File; import java.io.FileInputStream; @@ -39,8 +37,8 @@ import com.the_qa_company.qendpoint.core.exceptions.IllegalFormatException; import com.the_qa_company.qendpoint.core.listener.ProgressListener; import com.the_qa_company.qendpoint.core.options.HDTOptions; -import it.unimi.dsi.fastutil.io.FastBufferedInputStream; -import it.unimi.dsi.fastutil.io.FastBufferedOutputStream; +import org.spf4j.io.BufferedInputStream; +import org.spf4j.io.BufferedOutputStream; import com.the_qa_company.qendpoint.core.util.BitUtil; import com.the_qa_company.qendpoint.core.util.Mutable; import com.the_qa_company.qendpoint.core.util.crc.CRC32; @@ -118,7 +116,7 @@ public void load(Iterator it, long numentries, ProgressL ByteString previousStr = null; try { - try (OutputStream out = new FastBufferedOutputStream(new FileOutputStream(file))) { + try (OutputStream out = new BufferedOutputStream(new FileOutputStream(file))) { while (it.hasNext()) { ByteString str = ByteString.of(it.next()); @@ -163,7 +161,7 @@ public void load(Iterator it, long numentries, ProgressL byteOut.writeTo(out); } - try (InputStream in = new FastBufferedInputStream(new FileInputStream(file))) { + try (InputStream in = new BufferedInputStream(new FileInputStream(file))) { // Read block by block // Read packed data diff --git a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/dictionary/impl/section/PFCDictionarySectionMap.java b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/dictionary/impl/section/PFCDictionarySectionMap.java index ba5432286..918ded014 100644 --- a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/dictionary/impl/section/PFCDictionarySectionMap.java +++ b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/dictionary/impl/section/PFCDictionarySectionMap.java @@ -25,7 +25,7 @@ import com.the_qa_company.qendpoint.core.exceptions.IllegalFormatException; import com.the_qa_company.qendpoint.core.exceptions.NotImplementedException; import com.the_qa_company.qendpoint.core.listener.ProgressListener; -import it.unimi.dsi.fastutil.io.FastBufferedInputStream; +import org.spf4j.io.BufferedInputStream; import com.the_qa_company.qendpoint.core.util.io.BigMappedByteBuffer; import com.the_qa_company.qendpoint.core.compact.integer.VByte; import com.the_qa_company.qendpoint.core.compact.sequence.Sequence; @@ -41,7 +41,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.BufferedInputStream; import java.io.Closeable; import java.io.File; import java.io.FileInputStream; @@ -376,7 +375,7 @@ public void load(Iterator it, long count, ProgressListen @Override public void save(OutputStream output, ProgressListener listener) throws IOException { - InputStream in = new FastBufferedInputStream(new FileInputStream(f)); + InputStream in = new BufferedInputStream(new FileInputStream(f)); IOUtil.skip(in, startOffset); IOUtil.copyStream(in, output, endOffset - startOffset); in.close(); diff --git a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/hdt/HDTManagerImpl.java b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/hdt/HDTManagerImpl.java index b4098665a..11b3ea6a3 100644 --- a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/hdt/HDTManagerImpl.java +++ b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/hdt/HDTManagerImpl.java @@ -20,7 +20,7 @@ import com.the_qa_company.qendpoint.core.rdf.RDFParserCallback; import com.the_qa_company.qendpoint.core.rdf.RDFParserFactory; import com.the_qa_company.qendpoint.core.rdf.TripleWriter; -import it.unimi.dsi.fastutil.io.FastBufferedOutputStream; +import org.spf4j.io.BufferedOutputStream; import com.the_qa_company.qendpoint.core.triples.TripleString; import com.the_qa_company.qendpoint.core.util.BitUtil; import com.the_qa_company.qendpoint.core.util.Profiler; @@ -36,7 +36,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.BufferedOutputStream; import java.io.File; import java.io.IOException; import java.io.InputStream; @@ -255,7 +254,7 @@ public HDTResult doGenerateHDT(String rdfFileName, String baseURI, RDFNotation r InputStream stream = readIs.is(); try (InputStream is = checksumPath != null ? new CRCInputStream(stream, new CRC32()) : stream; - OutputStream os = new FastBufferedOutputStream( + OutputStream os = new BufferedOutputStream( Files.newOutputStream(preDownload, openOptions))) { IOUtil.copy(is, os, listener, 10_000_000); if (is instanceof CRCInputStream crcIs) { diff --git a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/hdt/impl/HDTImpl.java b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/hdt/impl/HDTImpl.java index e27279b5b..92c9742b6 100644 --- a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/hdt/impl/HDTImpl.java +++ b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/hdt/impl/HDTImpl.java @@ -52,8 +52,8 @@ import com.the_qa_company.qendpoint.core.options.HDTOptions; import com.the_qa_company.qendpoint.core.options.HDTOptionsKeys; import com.the_qa_company.qendpoint.core.options.HDTSpecification; -import it.unimi.dsi.fastutil.io.FastBufferedInputStream; -import it.unimi.dsi.fastutil.io.FastBufferedOutputStream; +import org.spf4j.io.BufferedInputStream; +import org.spf4j.io.BufferedOutputStream; import com.the_qa_company.qendpoint.core.triples.DictionaryEntriesDiff; import com.the_qa_company.qendpoint.core.triples.IteratorTripleID; import com.the_qa_company.qendpoint.core.triples.IteratorTripleString; @@ -76,8 +76,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.BufferedInputStream; -import java.io.BufferedOutputStream; import java.io.File; import java.io.FileInputStream; import java.io.FileNotFoundException; @@ -162,9 +160,9 @@ public void loadFromHDT(InputStream input, ProgressListener listener) throws IOE public void loadFromHDT(String hdtFileName, ProgressListener listener) throws IOException { InputStream in; if (hdtFileName.endsWith(".gz")) { - in = new FastBufferedInputStream(new GZIPInputStream(new FileInputStream(hdtFileName))); + in = new BufferedInputStream(new GZIPInputStream(new FileInputStream(hdtFileName))); } else { - in = new CountInputStream(new FastBufferedInputStream(new FileInputStream(hdtFileName))); + in = new CountInputStream(new BufferedInputStream(new FileInputStream(hdtFileName))); } loadFromHDT(in, listener); in.close(); @@ -194,8 +192,8 @@ public void mapFromHDT(File f, long offset, ProgressListener listener) throws IO } boolean dumpBinInfo = spec.getBoolean(HDTOptionsKeys.DUMP_BINARY_OFFSETS, false); - try (CountInputStream input = new CountInputStream( - new FastBufferedInputStream(new FileInputStream(hdtFileName)), dumpBinInfo)) { + try (CountInputStream input = new CountInputStream(new BufferedInputStream(new FileInputStream(hdtFileName)), + dumpBinInfo)) { input.printIndex("HDT CI"); @@ -258,7 +256,7 @@ public void mapFromHDT(File f, long offset, ProgressListener listener) throws IO */ @Override public void saveToHDT(String fileName, ProgressListener listener) throws IOException { - try (OutputStream out = new FastBufferedOutputStream(new FileOutputStream(fileName))) { + try (OutputStream out = new BufferedOutputStream(new FileOutputStream(fileName))) { // OutputStream out = new GZIPOutputStream(new // BufferedOutputStream(new FileOutputStream(fileName))); saveToHDT(out, listener); @@ -512,7 +510,7 @@ public void loadOrCreateIndex(ProgressListener listener, HDTOptions spec) throws } CountInputStream in = null; try { - in = new CountInputStream(new FastBufferedInputStream(new FileInputStream(ff))); + in = new CountInputStream(new BufferedInputStream(new FileInputStream(ff))); ci.load(in); if (isMapped) { triples.mapIndex(in, new File(indexName), ci, listener); @@ -532,7 +530,7 @@ public void loadOrCreateIndex(ProgressListener listener, HDTOptions spec) throws if (this.hdtFileName != null) { OutputStream out = null; try { - out = new FastBufferedOutputStream(new FileOutputStream(versionName)); + out = new BufferedOutputStream(new FileOutputStream(versionName)); ci.clear(); triples.saveIndex(out, ci, listener); out.close(); @@ -613,7 +611,7 @@ public void cat(String location, HDT hdt1, HDT hdt2, ProgressListener listener, // map the generated dictionary FourSectionDictionaryBig dictionary; try (CountInputStream fis = new CountInputStream( - new FastBufferedInputStream(new FileInputStream(location + "dictionary")))) { + new BufferedInputStream(new FileInputStream(location + "dictionary")))) { dictionary = new FourSectionDictionaryBig(new HDTSpecification()); fis.mark(1024); ci2.load(fis); @@ -658,7 +656,7 @@ public void cat(String location, HDT hdt1, HDT hdt2, ProgressListener listener, // map the triples try (CountInputStream fis2 = new CountInputStream( - new FastBufferedInputStream(new FileInputStream(location + "triples")))) { + new BufferedInputStream(new FileInputStream(location + "triples")))) { ControlInfo ci2 = new ControlInformation(); ci2.clear(); fis2.mark(1024); @@ -697,7 +695,7 @@ public void catCustom(String location, HDT hdt1, HDT hdt2, ProgressListener list // map the generated dictionary ControlInfo ci2 = new ControlInformation(); try (CountInputStream fis = new CountInputStream( - new FastBufferedInputStream(new FileInputStream(location + "dictionary")))) { + new BufferedInputStream(new FileInputStream(location + "dictionary")))) { HDTSpecification spec = new HDTSpecification(); spec.set(HDTOptionsKeys.TEMP_DICTIONARY_IMPL_KEY, HDTOptionsKeys.TEMP_DICTIONARY_IMPL_VALUE_MULT_HASH); spec.set(HDTOptionsKeys.DICTIONARY_TYPE_KEY, HDTOptionsKeys.DICTIONARY_TYPE_VALUE_MULTI_OBJECTS); @@ -760,7 +758,7 @@ public void catCustom(String location, HDT hdt1, HDT hdt2, ProgressListener list Files.delete(Paths.get(location + "O2" + "Types")); // map the triples try (CountInputStream fis2 = new CountInputStream( - new FastBufferedInputStream(new FileInputStream(location + "triples")))) { + new BufferedInputStream(new FileInputStream(location + "triples")))) { ControlInformation ci2 = new ControlInformation(); ci2.clear(); fis2.mark(1024); @@ -817,7 +815,7 @@ public void diffBit(String location, HDT hdt, Bitmap deleteBitmap, ProgressListe ControlInfo ci2 = new ControlInformation(); try (CountInputStream fis = new CountInputStream( - new FastBufferedInputStream(new FileInputStream(location + "dictionary")))) { + new BufferedInputStream(new FileInputStream(location + "dictionary")))) { fis.mark(1024); ci2.load(fis); fis.reset(); diff --git a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/hdt/impl/WriteHDTImpl.java b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/hdt/impl/WriteHDTImpl.java index 2f174bf00..fd481e10d 100644 --- a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/hdt/impl/WriteHDTImpl.java +++ b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/hdt/impl/WriteHDTImpl.java @@ -8,14 +8,13 @@ import com.the_qa_company.qendpoint.core.header.HeaderPrivate; import com.the_qa_company.qendpoint.core.listener.ProgressListener; import com.the_qa_company.qendpoint.core.options.HDTOptions; -import it.unimi.dsi.fastutil.io.FastBufferedOutputStream; +import org.spf4j.io.BufferedOutputStream; import com.the_qa_company.qendpoint.core.triples.IteratorTripleString; import com.the_qa_company.qendpoint.core.triples.TriplesPrivate; import com.the_qa_company.qendpoint.core.triples.impl.WriteBitmapTriples; import com.the_qa_company.qendpoint.core.util.io.CloseSuppressPath; import com.the_qa_company.qendpoint.core.util.io.IOUtil; -import java.io.BufferedOutputStream; import java.io.File; import java.io.IOException; import java.io.InputStream; @@ -88,7 +87,7 @@ public void loadOrCreateIndex(ProgressListener listener, HDTOptions disk) { @Override public void saveToHDT(String fileName, ProgressListener listener) throws IOException { - try (OutputStream out = new FastBufferedOutputStream(Files.newOutputStream(Path.of(fileName)))) { + try (OutputStream out = new BufferedOutputStream(Files.newOutputStream(Path.of(fileName)))) { saveToHDT(out, listener); } } diff --git a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/hdt/impl/diskimport/SectionCompressor.java b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/hdt/impl/diskimport/SectionCompressor.java index a183821b8..72bb8388c 100644 --- a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/hdt/impl/diskimport/SectionCompressor.java +++ b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/hdt/impl/diskimport/SectionCompressor.java @@ -60,7 +60,8 @@ public SectionCompressor(CloseSuppressPath baseFileName, AsyncIteratorFetcher implements Supplier { private final Iterator iterator; - private final Lock lock = new ReentrantLock(); private boolean end; - ConcurrentLinkedQueue queue = new ConcurrentLinkedQueue<>(); + private ConcurrentLinkedQueue queue = new ConcurrentLinkedQueue<>(); public AsyncIteratorFetcher(Iterator iterator) { this.iterator = iterator; @@ -27,7 +27,7 @@ public AsyncIteratorFetcher(Iterator iterator) { /** * @return an element from the iterator, this method is thread safe */ - @Override +// @Override // public E get() { // lock.lock(); // try { @@ -41,15 +41,20 @@ public AsyncIteratorFetcher(Iterator iterator) { // } // } + AtomicInteger counter = new AtomicInteger(0); + + @Override public E get() { E poll = queue.poll(); if (poll != null) { + counter.incrementAndGet(); return poll; } synchronized (this) { poll = queue.poll(); if (poll != null) { + counter.incrementAndGet(); return poll; } @@ -63,8 +68,12 @@ public E get() { } this.queue = newqueue; if (poll != null) { + counter.incrementAndGet(); return poll; } + + System.out.println("AsyncIteratorFetcher: " + counter.get()); + end = true; return null; } diff --git a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/iterator/utils/AsyncIteratorFetcherUnordered.java b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/iterator/utils/AsyncIteratorFetcherUnordered.java index 677a5c119..b946eac41 100644 --- a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/iterator/utils/AsyncIteratorFetcherUnordered.java +++ b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/iterator/utils/AsyncIteratorFetcherUnordered.java @@ -4,6 +4,7 @@ import java.util.ArrayList; import java.util.Iterator; import java.util.Queue; +import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.locks.Lock; import java.util.concurrent.locks.ReentrantLock; @@ -24,6 +25,8 @@ public class AsyncIteratorFetcherUnordered extends AsyncIteratorFetcher { new ArrayDeque(BUFFER), new ArrayDeque(BUFFER), new ArrayDeque(BUFFER), new ArrayDeque(BUFFER), new ArrayDeque(BUFFER), }; + AtomicInteger counter = new AtomicInteger(0); + public AsyncIteratorFetcherUnordered(Iterator iterator) { super(iterator); this.iterator = iterator; @@ -45,6 +48,7 @@ public E get() { E poll = eQueue.poll(); if (poll != null) { + counter.incrementAndGet(); return poll; } } @@ -61,6 +65,7 @@ public E get() { E poll = es.poll(); if (poll != null) { + counter.incrementAndGet(); return poll; } @@ -86,6 +91,7 @@ public E get() { if (poll == null) { queue[index] = null; } else { + counter.incrementAndGet(); return poll; } } @@ -101,12 +107,23 @@ public E get() { E poll = eQueue.poll(); if (poll != null) { + counter.incrementAndGet(); return poll; } } } } } + + synchronized (this) { + if (iterator.hasNext()) { + E poll = iterator.next(); + counter.incrementAndGet(); + return poll; + } + } + + System.out.println("AsyncIteratorFetcherUnordered: " + counter.get()); end = true; return null; diff --git a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/iterator/utils/ParallelMerge.java b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/iterator/utils/ParallelMerge.java index 35b2a5d56..ff7246187 100644 --- a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/iterator/utils/ParallelMerge.java +++ b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/iterator/utils/ParallelMerge.java @@ -269,7 +269,7 @@ private synchronized void setExhausted(Supplier exhaustedFlag) { // Ugly but easy: if exhaustedFlag points to leftExhausted, set it, else // set rightExhausted // A better design might store a boolean or do a callback. - com.github.jsonldjava.shaded.com.google.common.base.Supplier isLeftExhausted = this::isLeftExhausted; + Supplier isLeftExhausted = this::isLeftExhausted; if (exhaustedFlag == isLeftExhausted) { leftExhausted = true; } else { diff --git a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/iterator/utils/PipedCopyIteratorUnordered.java b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/iterator/utils/PipedCopyIteratorUnordered.java index 798d094e3..a7760d692 100644 --- a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/iterator/utils/PipedCopyIteratorUnordered.java +++ b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/iterator/utils/PipedCopyIteratorUnordered.java @@ -1,11 +1,11 @@ package com.the_qa_company.qendpoint.core.iterator.utils; -import java.io.Closeable; import java.io.IOException; import java.util.Iterator; import java.util.Objects; import java.util.concurrent.ArrayBlockingQueue; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; import java.util.function.Function; @@ -19,6 +19,8 @@ public class PipedCopyIteratorUnordered extends PipedCopyIterator { + private static final AtomicInteger counter = new AtomicInteger(0); + /** * RuntimeException generated by the PipedCopyIterator * @@ -126,14 +128,20 @@ public T get() { new ArrayBlockingQueue<>(16 * 1024), new ArrayBlockingQueue<>(16 * 1024), new ArrayBlockingQueue<>(16 * 1024), new ArrayBlockingQueue<>(16 * 1024), }; + private final AtomicBoolean[] queueEnd = new AtomicBoolean[queue.length]; + + { + for (int i = 0; i < queueEnd.length; i++) { + queueEnd[i] = new AtomicBoolean(false); + } + } + private T next; private boolean end; private PipedIteratorException exception; private Thread thread; - AtomicInteger indexHasNext = new AtomicInteger(0); - volatile ArrayBlockingQueue> focusQueue; @Override @@ -150,7 +158,6 @@ public boolean hasNext() { obj = useFocusQueue(); if (obj == null) { - obj = useThreadBasedQueue(); } @@ -158,7 +165,12 @@ public boolean hasNext() { throw new PipedIteratorException("Can't read pipe", e); } + if (obj == null || obj.end()) { + obj = checkAllQueues(obj); + } + if (obj.end()) { + System.out.println("End of queue: " + counter.get()); end = true; if (exception != null) { throw exception; @@ -175,18 +187,70 @@ private QueueObject useThreadBasedQueue() throws InterruptedException { obj = queue[i % queue.length].poll(); if (obj == null) { obj = iterateThroughAllQueues(obj); + } else if (obj.end()) { + setQueueEnd(queue[i % queue.length]); } else if (focusQueue == null) { focusQueue = queue[i % queue.length]; } return obj; } + private QueueObject checkAllQueues(QueueObject originalObj) { + QueueObject obj = null; + boolean done; + + do { + done = true; + for (int i = 0; i < queue.length; i++) { + if (queueEnd[i].get()) { + continue; + } + done = false; + ArrayBlockingQueue> queueObjects = queue[i]; + obj = queueObjects.poll(); + if (obj == null) { + try { + Thread.sleep(100); + } catch (InterruptedException e) { + throw new RuntimeException(e); + } + } else if (!obj.end()) { + return obj; + } else { + queueEnd[i].set(true); + } + } + } while (!done); + + if (obj == null) { + obj = originalObj; + } + + return obj; + } + + private boolean isEnd() { + for (AtomicBoolean atomicBoolean : queueEnd) { + if (!atomicBoolean.get()) { + return false; + } + } + + return true; + } + private QueueObject iterateThroughAllQueues(QueueObject obj) throws InterruptedException { while (obj == null) { - for (ArrayBlockingQueue> queueObjects : queue) { + for (int i = 0; i < queue.length; i++) { + if (queueEnd[i].get()) { + continue; + } + ArrayBlockingQueue> queueObjects = queue[i]; obj = queueObjects.poll(); if (obj != null) { - if (focusQueue == null) { + if (obj.end()) { + queueEnd[i].set(true); + } else if (focusQueue == null) { focusQueue = queueObjects; } return obj; @@ -204,6 +268,9 @@ private QueueObject useFocusQueue() throws InterruptedException { QueueObject poll = focusQueue.poll(); if (poll != null) { obj = poll; + if (obj.end()) { + setQueueEnd(focusQueue); + } } else { obj = null; this.focusQueue = null; @@ -214,11 +281,24 @@ private QueueObject useFocusQueue() throws InterruptedException { return obj; } + private void setQueueEnd(ArrayBlockingQueue> focusQueue) { + for (int i = 0; i < queue.length; i++) { + if (queue[i] == focusQueue) { + queueEnd[i].set(true); + break; + } + } + } + @Override public T next() { if (!hasNext()) { return null; } + int i = counter.incrementAndGet(); + if (i % 100 == 0 && i > 176451821) { + System.out.println("PipedCopyIteratorUnordered: " + i); + } T next = this.next; this.next = null; return next; diff --git a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/iterator/utils/SizeFetcher.java b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/iterator/utils/SizeFetcher.java index ac9320cfe..d775f5f83 100644 --- a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/iterator/utils/SizeFetcher.java +++ b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/iterator/utils/SizeFetcher.java @@ -30,7 +30,7 @@ public static SizeFetcher of(Supplier supplier, ToLongFunction size private final long maxSize; - private long size; + private volatile long size; public SizeFetcher(Supplier supplier, ToLongFunction sizeGetter, long maxSize) { this.supplier = supplier; @@ -50,6 +50,9 @@ public E get() { } size += sizeGetter.applyAsLong(e); +// if (size % 100000 == 0) { +// System.out.println("Size: " + size); +// } return e; } diff --git a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/options/ControlInfo.java b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/options/ControlInfo.java index cb6ab7dff..743804610 100644 --- a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/options/ControlInfo.java +++ b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/options/ControlInfo.java @@ -19,9 +19,8 @@ package com.the_qa_company.qendpoint.core.options; -import it.unimi.dsi.fastutil.io.FastBufferedOutputStream; +import org.spf4j.io.BufferedOutputStream; -import java.io.BufferedOutputStream; import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; @@ -45,7 +44,7 @@ enum Type { void setFormat(String format); default void save(Path filename) throws IOException { - try (OutputStream os = new FastBufferedOutputStream(Files.newOutputStream(filename))) { + try (OutputStream os = new BufferedOutputStream(Files.newOutputStream(filename))) { save(os); } } diff --git a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/options/HDTOptionsFile.java b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/options/HDTOptionsFile.java index cda54ffce..9c6d56201 100644 --- a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/options/HDTOptionsFile.java +++ b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/options/HDTOptionsFile.java @@ -2,15 +2,13 @@ import com.the_qa_company.qendpoint.core.compact.integer.VByte; import com.the_qa_company.qendpoint.core.listener.ProgressListener; -import it.unimi.dsi.fastutil.io.FastBufferedInputStream; -import it.unimi.dsi.fastutil.io.FastBufferedOutputStream; +import org.spf4j.io.BufferedInputStream; +import org.spf4j.io.BufferedOutputStream; import com.the_qa_company.qendpoint.core.util.crc.CRC32; import com.the_qa_company.qendpoint.core.util.crc.CRCInputStream; import com.the_qa_company.qendpoint.core.util.crc.CRCOutputStream; import com.the_qa_company.qendpoint.core.util.io.IOUtil; -import java.io.BufferedInputStream; -import java.io.BufferedOutputStream; import java.io.IOException; import java.nio.file.Files; import java.nio.file.Path; @@ -39,7 +37,7 @@ public void sync() throws IOException { return; } ProgressListener l = ProgressListener.ignore(); - try (CRCInputStream is = new CRCInputStream(new FastBufferedInputStream(Files.newInputStream(location)), + try (CRCInputStream is = new CRCInputStream(new BufferedInputStream(Files.newInputStream(location)), new CRC32())) { if (IOUtil.readLong(is) != MAGIC) throw new IOException("Can't read HDTOptions file: Bad magic"); @@ -61,7 +59,7 @@ public void sync() throws IOException { public void save() throws IOException { ProgressListener l = ProgressListener.ignore(); - try (CRCOutputStream os = new CRCOutputStream(new FastBufferedOutputStream(Files.newOutputStream(location)), + try (CRCOutputStream os = new CRCOutputStream(new BufferedOutputStream(Files.newOutputStream(location)), new CRC32())) { IOUtil.writeLong(os, MAGIC); Set keys = options.getKeys(); diff --git a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/rdf/parsers/RDFParserRIOT.java b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/rdf/parsers/RDFParserRIOT.java index 19214a3f0..963dff135 100644 --- a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/rdf/parsers/RDFParserRIOT.java +++ b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/rdf/parsers/RDFParserRIOT.java @@ -141,10 +141,17 @@ public static class ElemStringBuffer implements StreamRDF { public ElemStringBuffer(RDFCallback callback) { this.callback = callback; + + Runtime.getRuntime().addShutdownHook( + new Thread(() -> { System.out.println("Total triples parsed: " + counter.get()); })); } @Override public void triple(Triple parsedTriple) { + int i = counter.incrementAndGet(); + if (i % 100 == 0 && i > 177271352) { + System.out.println("Triple count: " + i); + } triple.setAll(JenaNodeFormatter.format(parsedTriple.getSubject()), JenaNodeFormatter.format(parsedTriple.getPredicate()), JenaNodeFormatter.format(parsedTriple.getObject())); @@ -153,6 +160,10 @@ public void triple(Triple parsedTriple) { @Override public void quad(Quad parsedQuad) { + int i = counter.incrementAndGet(); + if (i % 100 == 0 && i > 177271352) { + System.out.println("Quad count: " + i); + } quad.setAll(JenaNodeFormatter.format(parsedQuad.getSubject()), JenaNodeFormatter.format(parsedQuad.getPredicate()), JenaNodeFormatter.format(parsedQuad.getObject()), JenaNodeFormatter.format(parsedQuad.getGraph())); diff --git a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/storage/QEPMapIdSorter.java b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/storage/QEPMapIdSorter.java index a4b83df6a..f8a4d44cb 100644 --- a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/storage/QEPMapIdSorter.java +++ b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/storage/QEPMapIdSorter.java @@ -14,11 +14,9 @@ import com.the_qa_company.qendpoint.core.util.disk.LongArray; import com.the_qa_company.qendpoint.core.util.io.CloseSuppressPath; import com.the_qa_company.qendpoint.core.util.io.Closer; -import it.unimi.dsi.fastutil.io.FastBufferedInputStream; -import it.unimi.dsi.fastutil.io.FastBufferedOutputStream; +import org.spf4j.io.BufferedInputStream; +import org.spf4j.io.BufferedOutputStream; -import java.io.BufferedInputStream; -import java.io.BufferedOutputStream; import java.io.Closeable; import java.io.IOException; import java.io.InputStream; @@ -27,7 +25,6 @@ import java.nio.file.Path; import java.util.ArrayList; import java.util.Arrays; -import java.util.Collections; import java.util.Iterator; import java.util.List; import java.util.function.Supplier; @@ -47,7 +44,7 @@ public int compareTo(QEPMapIds o) { } public static final long MAX_ELEMENT_SIZE_THRESHOLD = 500_000_000L; // max - // 500MB + // 500MB private final LongArray ids; private long index; private final CloseSuppressPath computeLocation; @@ -96,7 +93,7 @@ public void sort() throws IOException { CloseSuppressPath output = merger.waitResult().orElse(null); if (output != null) { - try (InputStream stream = new FastBufferedInputStream(Files.newInputStream(output))) { + try (InputStream stream = new BufferedInputStream(Files.newInputStream(output))) { QEPMapReader reader = new QEPMapReader(stream); long index = 0; @@ -158,7 +155,7 @@ private record Merger(long chunkSize) implements KWayMerger.KWayMergerImpl flux, CloseSuppressPath output) throws KWayMerger.KWayMergerException { - try (OutputStream stream = new FastBufferedOutputStream(Files.newOutputStream(output))) { + try (OutputStream stream = new BufferedOutputStream(Files.newOutputStream(output))) { QEPMapIds ids; List idList = new ArrayList<>(); @@ -201,7 +198,7 @@ public void mergeChunks(List inputs, CloseSuppressPath output InputStream[] pathInput = new InputStream[inputs.size()]; for (int i = 0; i < pathInput.length; i++) { - pathInput[i] = new FastBufferedInputStream(Files.newInputStream(inputs.get(i))); + pathInput[i] = new BufferedInputStream(Files.newInputStream(inputs.get(i))); } try { @@ -209,7 +206,7 @@ public void mergeChunks(List inputs, CloseSuppressPath output ExceptionIterator tree = MergeExceptionIterator .buildOfTree(QEPMapReader::new, Arrays.asList(pathInput), 0, inputs.size()); - try (OutputStream stream = new FastBufferedOutputStream(Files.newOutputStream(output))) { + try (OutputStream stream = new BufferedOutputStream(Files.newOutputStream(output))) { while (tree.hasNext()) { QEPMapIds ids = tree.next(); VByte.encode(stream, ids.origin()); diff --git a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/triples/impl/BitmapTriplesCat.java b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/triples/impl/BitmapTriplesCat.java index 00c5ce629..419c7fc68 100644 --- a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/triples/impl/BitmapTriplesCat.java +++ b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/triples/impl/BitmapTriplesCat.java @@ -13,7 +13,6 @@ package com.the_qa_company.qendpoint.core.triples.impl; -import java.io.BufferedOutputStream; import java.io.FileOutputStream; import java.io.IOException; import java.io.OutputStream; @@ -26,7 +25,7 @@ import com.the_qa_company.qendpoint.core.listener.ProgressListener; import com.the_qa_company.qendpoint.core.options.ControlInfo; import com.the_qa_company.qendpoint.core.options.ControlInformation; -import it.unimi.dsi.fastutil.io.FastBufferedOutputStream; +import org.spf4j.io.BufferedOutputStream; import com.the_qa_company.qendpoint.core.triples.IteratorTripleID; import com.the_qa_company.qendpoint.core.triples.TripleID; import com.the_qa_company.qendpoint.core.util.BitUtil; @@ -120,7 +119,7 @@ public void cat(IteratorTripleID it, ProgressListener listener) throws IOExcepti vectorY.aggressiveTrimToSize(); vectorZ.trimToSize(); - try (OutputStream bos = new FastBufferedOutputStream(new FileOutputStream(location + "triples"))) { + try (OutputStream bos = new BufferedOutputStream(new FileOutputStream(location + "triples"))) { ControlInfo ci = new ControlInformation(); ci.setType(ControlInfo.Type.TRIPLES); ci.setFormat(HDTVocabulary.TRIPLES_TYPE_BITMAP); diff --git a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/triples/impl/BitmapTriplesIndexFile.java b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/triples/impl/BitmapTriplesIndexFile.java index d672b6b9c..f0298340b 100644 --- a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/triples/impl/BitmapTriplesIndexFile.java +++ b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/triples/impl/BitmapTriplesIndexFile.java @@ -19,8 +19,8 @@ import com.the_qa_company.qendpoint.core.listener.ProgressListener; import com.the_qa_company.qendpoint.core.options.HDTOptions; import com.the_qa_company.qendpoint.core.options.HDTOptionsKeys; -import it.unimi.dsi.fastutil.io.FastBufferedInputStream; -import it.unimi.dsi.fastutil.io.FastBufferedOutputStream; +import org.spf4j.io.BufferedInputStream; +import org.spf4j.io.BufferedOutputStream; import com.the_qa_company.qendpoint.core.triples.TripleID; import com.the_qa_company.qendpoint.core.util.BitUtil; import com.the_qa_company.qendpoint.core.util.concurrent.KWayMerger; @@ -33,8 +33,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.BufferedInputStream; -import java.io.BufferedOutputStream; import java.io.Closeable; import java.io.IOException; import java.io.InterruptedIOException; @@ -132,7 +130,7 @@ public static BitmapTriplesIndex map(Path file, FileChannel channel, BitmapTripl } } - CountInputStream stream = new CountInputStream(new FastBufferedInputStream(Channels.newInputStream(channel))); + CountInputStream stream = new CountInputStream(new BufferedInputStream(Channels.newInputStream(channel))); stream.skipNBytes(headerSize); String orderCfg = IOUtil.readSizedString(stream, ProgressListener.ignore()); @@ -317,7 +315,7 @@ public static void generateIndex(BitmapTriples triples, Path destination, Triple seqZ.trimToSize(); // saving the index - try (OutputStream output = new FastBufferedOutputStream(Files.newOutputStream(destination))) { + try (OutputStream output = new BufferedOutputStream(Files.newOutputStream(destination))) { output.write(MAGIC); IOUtil.writeLong(output, signature(triples)); diff --git a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/util/LargeFakeDataSetStreamSupplier.java b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/util/LargeFakeDataSetStreamSupplier.java index 1a18b7f93..ef45a1d04 100644 --- a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/util/LargeFakeDataSetStreamSupplier.java +++ b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/util/LargeFakeDataSetStreamSupplier.java @@ -7,14 +7,13 @@ import com.the_qa_company.qendpoint.core.iterator.utils.MapIterator; import com.the_qa_company.qendpoint.core.options.HDTOptions; import com.the_qa_company.qendpoint.core.quad.QuadString; -import it.unimi.dsi.fastutil.io.FastBufferedOutputStream; +import org.spf4j.io.BufferedOutputStream; import com.the_qa_company.qendpoint.core.triples.TripleString; import com.the_qa_company.qendpoint.core.util.concurrent.ExceptionThread; import com.the_qa_company.qendpoint.core.util.string.ByteStringUtil; import org.apache.commons.compress.compressors.bzip2.BZip2CompressorOutputStream; import org.apache.commons.compress.compressors.xz.XZCompressorOutputStream; -import java.io.BufferedOutputStream; import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; @@ -195,7 +194,7 @@ public void createNTFile(Path file) throws IOException { */ public void createNTFile(Path file, CompressionType compressionType) throws IOException { try (Writer writer = new OutputStreamWriter( - new FastBufferedOutputStream(compressionType.compress(Files.newOutputStream(file))))) { + new BufferedOutputStream(compressionType.compress(Files.newOutputStream(file))))) { createNTFile(writer); } } diff --git a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/util/Profiler.java b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/util/Profiler.java index 9139d824f..4648a7254 100644 --- a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/util/Profiler.java +++ b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/util/Profiler.java @@ -2,11 +2,9 @@ import com.the_qa_company.qendpoint.core.options.HDTOptions; import com.the_qa_company.qendpoint.core.options.HDTOptionsKeys; -import it.unimi.dsi.fastutil.io.FastBufferedInputStream; -import it.unimi.dsi.fastutil.io.FastBufferedOutputStream; +import org.spf4j.io.BufferedInputStream; +import org.spf4j.io.BufferedOutputStream; -import java.io.BufferedInputStream; -import java.io.BufferedOutputStream; import java.io.EOFException; import java.io.IOException; import java.io.InputStream; @@ -50,7 +48,7 @@ public static Profiler getProfilerById(long id) { */ public static Profiler readFromDisk(Path inputPath) throws IOException { Profiler p = new Profiler(""); - try (InputStream is = new FastBufferedInputStream(Files.newInputStream(inputPath))) { + try (InputStream is = new BufferedInputStream(Files.newInputStream(inputPath))) { for (byte b : HEADER) { if (is.read() != b) { throw new IOException("Missing header for the profiling file!"); @@ -288,7 +286,7 @@ public void writeProfiling() throws IOException { * @param outputPath output path */ public void writeToDisk(Path outputPath) throws IOException { - try (OutputStream os = new FastBufferedOutputStream(Files.newOutputStream(outputPath))) { + try (OutputStream os = new BufferedOutputStream(Files.newOutputStream(outputPath))) { for (byte b : HEADER) { os.write(b); } diff --git a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/util/RDFInfo.java b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/util/RDFInfo.java index 728890f0e..92a2ad623 100644 --- a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/util/RDFInfo.java +++ b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/util/RDFInfo.java @@ -26,7 +26,7 @@ import com.the_qa_company.qendpoint.core.exceptions.ParserException; import com.the_qa_company.qendpoint.core.options.HDTOptions; import com.the_qa_company.qendpoint.core.rdf.RDFParserCallback; -import it.unimi.dsi.fastutil.io.FastBufferedInputStream; +import org.spf4j.io.BufferedInputStream; /** * A class for getting basic information about a file @@ -136,7 +136,7 @@ public static float getCompression(HDTOptions specs) { */ public static long countLines(String filename, RDFParserCallback parser, RDFNotation notation) throws IOException, ParserException { - InputStream is = new FastBufferedInputStream(new FileInputStream(filename)); + InputStream is = new BufferedInputStream(new FileInputStream(filename)); try { byte[] c = new byte[1024]; int count = 0; diff --git a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/util/io/CloseSuppressPath.java b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/util/io/CloseSuppressPath.java index 92aed90c8..eb8ab586b 100644 --- a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/util/io/CloseSuppressPath.java +++ b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/util/io/CloseSuppressPath.java @@ -1,10 +1,8 @@ package com.the_qa_company.qendpoint.core.util.io; -import it.unimi.dsi.fastutil.io.FastBufferedInputStream; -import it.unimi.dsi.fastutil.io.FastBufferedOutputStream; +import org.spf4j.io.BufferedInputStream; +import org.spf4j.io.BufferedOutputStream; -import java.io.BufferedInputStream; -import java.io.BufferedOutputStream; import java.io.Closeable; import java.io.File; import java.io.IOException; @@ -235,7 +233,7 @@ public Spliterator spliterator() { } public InputStream openInputStream(int bufferSize, OpenOption... options) throws IOException { - return new FastBufferedInputStream(openInputStream(options), bufferSize); + return new BufferedInputStream(openInputStream(options), bufferSize); } public InputStream openInputStream(OpenOption... options) throws IOException { @@ -247,7 +245,7 @@ private OutputStream openOutputStream(OpenOption... options) throws IOException } public OutputStream openOutputStream(int bufferSize, OpenOption... options) throws IOException { - return new FastBufferedOutputStream(openOutputStream(options), bufferSize); + return new BufferedOutputStream(openOutputStream(options), bufferSize); } /** diff --git a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/util/io/IOUtil.java b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/util/io/IOUtil.java index 3d5ea468c..2351fd338 100644 --- a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/util/io/IOUtil.java +++ b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/util/io/IOUtil.java @@ -25,7 +25,7 @@ import com.the_qa_company.qendpoint.core.listener.ProgressListener; import com.the_qa_company.qendpoint.core.options.HDTOptions; import com.the_qa_company.qendpoint.core.options.HDTOptionsKeys; -import it.unimi.dsi.fastutil.io.FastBufferedInputStream; +import org.spf4j.io.BufferedInputStream; import com.the_qa_company.qendpoint.core.unsafe.MemoryUtils; import com.the_qa_company.qendpoint.core.unsafe.UnsafeLongArray; import com.the_qa_company.qendpoint.core.util.StringUtil; @@ -34,7 +34,6 @@ import org.apache.commons.compress.compressors.bzip2.BZip2CompressorInputStream; import org.apache.commons.compress.compressors.xz.XZCompressorInputStream; -import java.io.BufferedInputStream; import java.io.BufferedReader; import java.io.ByteArrayOutputStream; import java.io.Closeable; @@ -395,9 +394,9 @@ public static InputStream getFileInputStream(String fileName, boolean uncompress con.connect(); input = con.getInputStream(); } else if (name.equals("-")) { - input = new FastBufferedInputStream(System.in); + input = new BufferedInputStream(System.in); } else { - input = new FastBufferedInputStream(new FileInputStream(fileName)); + input = new BufferedInputStream(new FileInputStream(fileName)); } if (!skipHandled) { input.skipNBytes(startLen); @@ -616,7 +615,7 @@ public static void copy(InputStream is, OutputStream os, ProgressListener pl, lo } public static void decompressGzip(File src, File trgt) throws IOException { - try (InputStream in = new GZIPInputStream(new FastBufferedInputStream(new FileInputStream(src)))) { + try (InputStream in = new GZIPInputStream(new BufferedInputStream(new FileInputStream(src)))) { Files.copy(in, trgt.toPath()); } } diff --git a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/util/io/compress/CompressNodeWriter.java b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/util/io/compress/CompressNodeWriter.java index 573ac8411..ee43534af 100644 --- a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/util/io/compress/CompressNodeWriter.java +++ b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/util/io/compress/CompressNodeWriter.java @@ -51,6 +51,7 @@ public void writeCRC() throws IOException { @Override public void close() throws IOException { writeCRC(); + out.flush(); out.close(); } } diff --git a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/util/nsd/NamespaceData.java b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/util/nsd/NamespaceData.java index f912ef239..176b111ed 100644 --- a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/util/nsd/NamespaceData.java +++ b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/util/nsd/NamespaceData.java @@ -3,16 +3,14 @@ import com.the_qa_company.qendpoint.core.compact.integer.VByte; import com.the_qa_company.qendpoint.core.listener.ProgressListener; import com.the_qa_company.qendpoint.core.storage.QEPCoreException; -import it.unimi.dsi.fastutil.io.FastBufferedInputStream; -import it.unimi.dsi.fastutil.io.FastBufferedOutputStream; +import org.spf4j.io.BufferedInputStream; +import org.spf4j.io.BufferedOutputStream; import com.the_qa_company.qendpoint.core.util.crc.CRC32; import com.the_qa_company.qendpoint.core.util.crc.CRCOutputStream; import com.the_qa_company.qendpoint.core.util.io.IOUtil; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.BufferedInputStream; -import java.io.BufferedOutputStream; import java.io.EOFException; import java.io.IOException; import java.io.InputStream; @@ -142,7 +140,7 @@ public void sync() throws QEPCoreException { */ public void load() throws QEPCoreException { synchronized (syncObject) { - try (InputStream is = new FastBufferedInputStream(Files.newInputStream(location))) { + try (InputStream is = new BufferedInputStream(Files.newInputStream(location))) { // check the binary magic byte[] header = is.readNBytes(MAGIC.length + 1); if (header.length < MAGIC.length + 1) { @@ -195,7 +193,7 @@ public void save(boolean onlyIfUpdated) throws QEPCoreException { return; // not updated } ProgressListener pl = ProgressListener.ignore(); - try (OutputStream osh = new FastBufferedOutputStream(Files.newOutputStream(location))) { + try (OutputStream osh = new BufferedOutputStream(Files.newOutputStream(location))) { // write magic and version osh.write(MAGIC); osh.write(NS_VERSION); diff --git a/qendpoint-core/src/test/java/com/the_qa_company/qendpoint/core/compact/bitmap/MultiRoaringBitmapTest.java b/qendpoint-core/src/test/java/com/the_qa_company/qendpoint/core/compact/bitmap/MultiRoaringBitmapTest.java index 71cd2fb49..cc7eec930 100644 --- a/qendpoint-core/src/test/java/com/the_qa_company/qendpoint/core/compact/bitmap/MultiRoaringBitmapTest.java +++ b/qendpoint-core/src/test/java/com/the_qa_company/qendpoint/core/compact/bitmap/MultiRoaringBitmapTest.java @@ -1,8 +1,8 @@ package com.the_qa_company.qendpoint.core.compact.bitmap; import com.the_qa_company.qendpoint.core.listener.ProgressListener; -import it.unimi.dsi.fastutil.io.FastBufferedInputStream; -import it.unimi.dsi.fastutil.io.FastBufferedOutputStream; +import org.spf4j.io.BufferedInputStream; +import org.spf4j.io.BufferedOutputStream; import com.the_qa_company.qendpoint.core.util.io.AbstractMapMemoryTest; import org.apache.commons.io.file.PathUtils; import org.junit.After; @@ -11,8 +11,6 @@ import org.junit.Test; import org.junit.rules.TemporaryFolder; -import java.io.BufferedInputStream; -import java.io.BufferedOutputStream; import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; @@ -56,7 +54,7 @@ public void serialSyncTest() throws IOException { map.set(0, 90, true); } - try (InputStream stream = new FastBufferedInputStream(Files.newInputStream(output)); + try (InputStream stream = new BufferedInputStream(Files.newInputStream(output)); MultiRoaringBitmap map = MultiRoaringBitmap.load(stream)) { for (int i = 0; i < 100; i++) { switch (i) { @@ -106,7 +104,7 @@ public void largeSerialSyncTest() throws IOException { assertTrue(map.access(layer, position)); } - try (OutputStream out = new FastBufferedOutputStream(Files.newOutputStream(output))) { + try (OutputStream out = new BufferedOutputStream(Files.newOutputStream(output))) { map.save(out, ProgressListener.ignore()); } } diff --git a/qendpoint-core/src/test/java/com/the_qa_company/qendpoint/core/dictionary/impl/kcat/KCatMergerTest.java b/qendpoint-core/src/test/java/com/the_qa_company/qendpoint/core/dictionary/impl/kcat/KCatMergerTest.java index b21cb0ae9..5028a9234 100644 --- a/qendpoint-core/src/test/java/com/the_qa_company/qendpoint/core/dictionary/impl/kcat/KCatMergerTest.java +++ b/qendpoint-core/src/test/java/com/the_qa_company/qendpoint/core/dictionary/impl/kcat/KCatMergerTest.java @@ -20,8 +20,8 @@ import com.the_qa_company.qendpoint.core.listener.ProgressListener; import com.the_qa_company.qendpoint.core.options.HDTOptions; import com.the_qa_company.qendpoint.core.options.HDTOptionsKeys; -import it.unimi.dsi.fastutil.io.FastBufferedInputStream; -import it.unimi.dsi.fastutil.io.FastBufferedOutputStream; +import org.spf4j.io.BufferedInputStream; +import org.spf4j.io.BufferedOutputStream; import com.the_qa_company.qendpoint.core.triples.IteratorTripleString; import com.the_qa_company.qendpoint.core.triples.TripleString; import com.the_qa_company.qendpoint.core.util.LargeFakeDataSetStreamSupplier; @@ -40,8 +40,6 @@ import org.junit.runners.Parameterized; import org.junit.runners.Suite; -import java.io.BufferedInputStream; -import java.io.BufferedOutputStream; import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; @@ -149,7 +147,7 @@ public void mergerTest() throws ParserException, IOException, InterruptedExcepti merger.startMerger(); // create DictionaryPrivate dict = merger.buildDictionary(); - try (OutputStream stream = new FastBufferedOutputStream(Files.newOutputStream(dictFile))) { + try (OutputStream stream = new BufferedOutputStream(Files.newOutputStream(dictFile))) { writeSection(dict.getShared(), stream); writeSection(dict.getSubjects(), stream); writeSection(dict.getPredicates(), stream); @@ -240,7 +238,7 @@ public void mergerTest() throws ParserException, IOException, InterruptedExcepti } } } - try (InputStream stream = new FastBufferedInputStream(Files.newInputStream(dictFile))) { + try (InputStream stream = new BufferedInputStream(Files.newInputStream(dictFile))) { // read the sections try (DictionarySection sh = loadSection(stream); DictionarySection su = loadSection(stream); @@ -483,7 +481,7 @@ public void diffMergerTest() throws ParserException, IOException, InterruptedExc merger.startMerger(); // create DictionaryPrivate dict = merger.buildDictionary(); - try (OutputStream stream = new FastBufferedOutputStream(Files.newOutputStream(dictFile))) { + try (OutputStream stream = new BufferedOutputStream(Files.newOutputStream(dictFile))) { writeSection(dict.getShared(), stream); writeSection(dict.getSubjects(), stream); writeSection(dict.getPredicates(), stream); @@ -501,7 +499,7 @@ public void diffMergerTest() throws ParserException, IOException, InterruptedExc } } } - try (InputStream stream = new FastBufferedInputStream(Files.newInputStream(dictFile))) { + try (InputStream stream = new BufferedInputStream(Files.newInputStream(dictFile))) { // read the sections try (DictionarySection sh = loadSection(stream); DictionarySection su = loadSection(stream); diff --git a/qendpoint-core/src/test/java/com/the_qa_company/qendpoint/core/dictionary/impl/section/WriteDictionarySectionTest.java b/qendpoint-core/src/test/java/com/the_qa_company/qendpoint/core/dictionary/impl/section/WriteDictionarySectionTest.java index 55cd7ac06..c4fa6e228 100644 --- a/qendpoint-core/src/test/java/com/the_qa_company/qendpoint/core/dictionary/impl/section/WriteDictionarySectionTest.java +++ b/qendpoint-core/src/test/java/com/the_qa_company/qendpoint/core/dictionary/impl/section/WriteDictionarySectionTest.java @@ -3,7 +3,7 @@ import com.the_qa_company.qendpoint.core.iterator.utils.MapIterator; import com.the_qa_company.qendpoint.core.listener.ProgressListener; import com.the_qa_company.qendpoint.core.options.HDTOptions; -import it.unimi.dsi.fastutil.io.FastBufferedOutputStream; +import org.spf4j.io.BufferedOutputStream; import com.the_qa_company.qendpoint.core.util.LargeFakeDataSetStreamSupplier; import com.the_qa_company.qendpoint.core.util.string.ByteString; import org.apache.commons.io.file.PathUtils; @@ -12,7 +12,6 @@ import org.junit.Test; import org.junit.rules.TemporaryFolder; -import java.io.BufferedOutputStream; import java.io.IOException; import java.io.OutputStream; import java.nio.file.Files; @@ -50,10 +49,10 @@ public void appenderTest() throws IOException { } Path t1Save = dir.resolve("t1.save"); Path t2Save = dir.resolve("t2.save"); - try (OutputStream os = new FastBufferedOutputStream(Files.newOutputStream(t1Save))) { + try (OutputStream os = new BufferedOutputStream(Files.newOutputStream(t1Save))) { section1.save(os, ProgressListener.ignore()); } - try (OutputStream os = new FastBufferedOutputStream(Files.newOutputStream(t2Save))) { + try (OutputStream os = new BufferedOutputStream(Files.newOutputStream(t2Save))) { section2.save(os, ProgressListener.ignore()); } diff --git a/qendpoint-core/src/test/java/com/the_qa_company/qendpoint/core/hdt/impl/HasmacHDTImporterTest.java b/qendpoint-core/src/test/java/com/the_qa_company/qendpoint/core/hdt/impl/HasmacHDTImporterTest.java index f3262fa3f..002048d01 100644 --- a/qendpoint-core/src/test/java/com/the_qa_company/qendpoint/core/hdt/impl/HasmacHDTImporterTest.java +++ b/qendpoint-core/src/test/java/com/the_qa_company/qendpoint/core/hdt/impl/HasmacHDTImporterTest.java @@ -27,8 +27,6 @@ import java.util.List; import java.util.concurrent.atomic.LongAdder; -import static org.apache.jena.riot.lang.extra.TurtleJCC.lang; - public class HasmacHDTImporterTest { private final HDTSpecification spec; diff --git a/qendpoint-core/src/test/java/com/the_qa_company/qendpoint/core/rdf/parsers/AbstractNTriplesParserTest.java b/qendpoint-core/src/test/java/com/the_qa_company/qendpoint/core/rdf/parsers/AbstractNTriplesParserTest.java index 374b6f41e..27549fed8 100644 --- a/qendpoint-core/src/test/java/com/the_qa_company/qendpoint/core/rdf/parsers/AbstractNTriplesParserTest.java +++ b/qendpoint-core/src/test/java/com/the_qa_company/qendpoint/core/rdf/parsers/AbstractNTriplesParserTest.java @@ -37,7 +37,7 @@ public void testIriUnescape() throws Exception { String input = format(CharSpace.ASCII, new Producer() { @Override public void writeTo(StreamRDF out) { - out.triple(new Triple(n, n, n)); + out.triple(Triple.create(n, n, n)); } }); @@ -60,9 +60,9 @@ public void testStringUnescape() throws Exception { String input = format(CharSpace.ASCII, new Producer() { @Override public void writeTo(StreamRDF out) { - out.triple(new Triple(s, p, o1)); - out.triple(new Triple(s, p, o2)); - out.triple(new Triple(s, p, o3)); + out.triple(Triple.create(s, p, o1)); + out.triple(Triple.create(s, p, o2)); + out.triple(Triple.create(s, p, o3)); } }); diff --git a/qendpoint-core/src/test/java/com/the_qa_company/qendpoint/core/rdf/parsers/TarTest.java b/qendpoint-core/src/test/java/com/the_qa_company/qendpoint/core/rdf/parsers/TarTest.java index 2257ed0ad..51341873f 100644 --- a/qendpoint-core/src/test/java/com/the_qa_company/qendpoint/core/rdf/parsers/TarTest.java +++ b/qendpoint-core/src/test/java/com/the_qa_company/qendpoint/core/rdf/parsers/TarTest.java @@ -1,11 +1,10 @@ package com.the_qa_company.qendpoint.core.rdf.parsers; -import java.io.BufferedInputStream; import java.io.FileInputStream; import java.io.InputStream; import java.util.zip.GZIPInputStream; -import it.unimi.dsi.fastutil.io.FastBufferedInputStream; +import org.spf4j.io.BufferedInputStream; import org.apache.commons.compress.archivers.tar.TarArchiveEntry; import org.apache.commons.compress.archivers.tar.TarArchiveInputStream; import com.the_qa_company.qendpoint.core.util.io.CountInputStream; @@ -15,9 +14,9 @@ public class TarTest { public static void main(String[] args) throws Throwable { - InputStream input = new CountInputStream(new FastBufferedInputStream( + InputStream input = new CountInputStream(new BufferedInputStream( new GZIPInputStream(new FileInputStream("/Users/mck/rdf/dataset/tgztest.tar.gz")))); -// InputStream input = new CountInputStream(new FastBufferedInputStream(new FileInputStream("/Users/mck/rdf/dataset/tgztest.tar"))); +// InputStream input = new CountInputStream(new BufferedInputStream(new FileInputStream("/Users/mck/rdf/dataset/tgztest.tar"))); final TarArchiveInputStream debInputStream = new TarArchiveInputStream(input); TarArchiveEntry entry; diff --git a/qendpoint-core/src/test/java/com/the_qa_company/qendpoint/core/triples/impl/BitmapQuadTriplesTest.java b/qendpoint-core/src/test/java/com/the_qa_company/qendpoint/core/triples/impl/BitmapQuadTriplesTest.java index 21934a22e..eb230e06b 100644 --- a/qendpoint-core/src/test/java/com/the_qa_company/qendpoint/core/triples/impl/BitmapQuadTriplesTest.java +++ b/qendpoint-core/src/test/java/com/the_qa_company/qendpoint/core/triples/impl/BitmapQuadTriplesTest.java @@ -4,8 +4,8 @@ import com.the_qa_company.qendpoint.core.enums.TripleComponentOrder; import com.the_qa_company.qendpoint.core.listener.ProgressListener; import com.the_qa_company.qendpoint.core.options.ControlInformation; -import it.unimi.dsi.fastutil.io.FastBufferedInputStream; -import it.unimi.dsi.fastutil.io.FastBufferedOutputStream; +import org.spf4j.io.BufferedInputStream; +import org.spf4j.io.BufferedOutputStream; import com.the_qa_company.qendpoint.core.triples.IteratorTripleID; import com.the_qa_company.qendpoint.core.triples.TripleID; import com.the_qa_company.qendpoint.core.util.io.AbstractMapMemoryTest; @@ -15,8 +15,6 @@ import org.junit.Test; import org.junit.rules.TemporaryFolder; -import java.io.BufferedInputStream; -import java.io.BufferedOutputStream; import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; @@ -147,14 +145,14 @@ public void triplesTest() throws IOException { assertEquals(it2.next(), it.next()); } - try (OutputStream stream = new FastBufferedOutputStream(Files.newOutputStream(path))) { + try (OutputStream stream = new BufferedOutputStream(Files.newOutputStream(path))) { triples.save(stream, new ControlInformation(), ProgressListener.ignore()); } } // load try (BitmapQuadTriples triples = new BitmapQuadTriples()) { - try (InputStream stream = new FastBufferedInputStream(Files.newInputStream(path))) { + try (InputStream stream = new BufferedInputStream(Files.newInputStream(path))) { ControlInformation ci = new ControlInformation(); ci.load(stream); triples.load(stream, ci, ProgressListener.ignore()); @@ -170,7 +168,7 @@ public void triplesTest() throws IOException { // map try (BitmapQuadTriples triples = new BitmapQuadTriples()) { - try (InputStream stream = new FastBufferedInputStream(Files.newInputStream(path))) { + try (InputStream stream = new BufferedInputStream(Files.newInputStream(path))) { CountInputStream cstream = new CountInputStream(stream); triples.mapFromFile(cstream, path.toFile(), ProgressListener.ignore()); } diff --git a/qendpoint-core/src/test/java/com/the_qa_company/qendpoint/core/triples/impl/utils/HDTTestUtils.java b/qendpoint-core/src/test/java/com/the_qa_company/qendpoint/core/triples/impl/utils/HDTTestUtils.java index aac5872da..ea4e84832 100644 --- a/qendpoint-core/src/test/java/com/the_qa_company/qendpoint/core/triples/impl/utils/HDTTestUtils.java +++ b/qendpoint-core/src/test/java/com/the_qa_company/qendpoint/core/triples/impl/utils/HDTTestUtils.java @@ -16,7 +16,6 @@ import com.the_qa_company.qendpoint.core.triples.TripleString; import com.the_qa_company.qendpoint.core.util.string.CharSequenceComparator; -import java.io.BufferedOutputStream; import java.io.BufferedWriter; import java.io.Closeable; import java.io.File; diff --git a/qendpoint-core/src/test/java/com/the_qa_company/qendpoint/core/util/LargeFakeDataSetStreamSupplierTest.java b/qendpoint-core/src/test/java/com/the_qa_company/qendpoint/core/util/LargeFakeDataSetStreamSupplierTest.java index c52fc7af6..a4a392590 100644 --- a/qendpoint-core/src/test/java/com/the_qa_company/qendpoint/core/util/LargeFakeDataSetStreamSupplierTest.java +++ b/qendpoint-core/src/test/java/com/the_qa_company/qendpoint/core/util/LargeFakeDataSetStreamSupplierTest.java @@ -10,7 +10,7 @@ import com.the_qa_company.qendpoint.core.options.HDTSpecification; import com.the_qa_company.qendpoint.core.rdf.RDFParserCallback; import com.the_qa_company.qendpoint.core.rdf.RDFParserFactory; -import it.unimi.dsi.fastutil.io.FastBufferedInputStream; +import org.spf4j.io.BufferedInputStream; import com.the_qa_company.qendpoint.core.triples.TripleString; import com.the_qa_company.qendpoint.core.triples.impl.utils.HDTTestUtils; import org.junit.Rule; @@ -20,7 +20,6 @@ import com.the_qa_company.qendpoint.core.iterator.utils.CombinedIterator; import com.the_qa_company.qendpoint.core.iterator.utils.PipedCopyIterator; -import java.io.BufferedInputStream; import java.io.BufferedWriter; import java.io.IOException; import java.io.InputStream; @@ -185,8 +184,8 @@ public void mergeTest() throws IOException, ParserException, NotFoundException { RDFParserCallback parser = RDFParserFactory.getParserCallback(RDFNotation.NTRIPLES, HDTOptions.of(Map.of(HDTOptionsKeys.NT_SIMPLE_PARSER_KEY, "true"))); try { - try (InputStream stream = new FastBufferedInputStream(Files.newInputStream(p12)); - InputStream stream2 = new FastBufferedInputStream(Files.newInputStream(p3)); + try (InputStream stream = new BufferedInputStream(Files.newInputStream(p12)); + InputStream stream2 = new BufferedInputStream(Files.newInputStream(p3)); PipedCopyIterator it1 = RDFParserFactory.readAsIterator(parser, stream, "http://w", true, RDFNotation.NTRIPLES); PipedCopyIterator it2 = RDFParserFactory.readAsIterator(parser, stream2, "http://w", diff --git a/qendpoint-store/src/main/java/com/the_qa_company/qendpoint/compiler/SailCompiler.java b/qendpoint-store/src/main/java/com/the_qa_company/qendpoint/compiler/SailCompiler.java index 498c15311..8ced05fe5 100644 --- a/qendpoint-store/src/main/java/com/the_qa_company/qendpoint/compiler/SailCompiler.java +++ b/qendpoint-store/src/main/java/com/the_qa_company/qendpoint/compiler/SailCompiler.java @@ -8,7 +8,7 @@ import com.the_qa_company.qendpoint.compiler.source.EmptyTripleSourceGetter; import com.the_qa_company.qendpoint.compiler.source.ModelTripleSourceGetter; import com.the_qa_company.qendpoint.compiler.source.SailTripleSourceModel; -import it.unimi.dsi.fastutil.io.FastBufferedInputStream; +import org.spf4j.io.BufferedInputStream; import com.the_qa_company.qendpoint.utils.sail.linked.LinkedSail; import org.eclipse.rdf4j.common.iteration.CloseableIteration; import org.eclipse.rdf4j.model.IRI; @@ -24,7 +24,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.BufferedInputStream; import java.io.IOException; import java.io.InputStream; import java.lang.reflect.InvocationTargetException; @@ -134,7 +133,7 @@ public void load(Path rdfFile) throws IOException { RDFFormat format = Rio.getParserFormatForFileName(rdfFile.getFileName().toString()) .orElseThrow(() -> new IllegalArgumentException("Can't find parser for file: " + rdfFile)); - try (InputStream stream = new FastBufferedInputStream(Files.newInputStream(rdfFile))) { + try (InputStream stream = new BufferedInputStream(Files.newInputStream(rdfFile))) { load(stream, format); } } diff --git a/qendpoint-store/src/main/java/com/the_qa_company/qendpoint/store/EndpointStore.java b/qendpoint-store/src/main/java/com/the_qa_company/qendpoint/store/EndpointStore.java index c7e4f8924..9716900c4 100644 --- a/qendpoint-store/src/main/java/com/the_qa_company/qendpoint/store/EndpointStore.java +++ b/qendpoint-store/src/main/java/com/the_qa_company/qendpoint/store/EndpointStore.java @@ -19,7 +19,7 @@ import com.the_qa_company.qendpoint.model.HDTValue; import com.the_qa_company.qendpoint.utils.BitArrayDisk; import com.the_qa_company.qendpoint.utils.CloseSafeHDT; -import it.unimi.dsi.fastutil.io.FastBufferedOutputStream; +import org.spf4j.io.BufferedOutputStream; import org.apache.commons.io.FileUtils; import org.apache.commons.io.file.PathUtils; import org.eclipse.rdf4j.common.concurrent.locks.Lock; @@ -51,7 +51,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.BufferedOutputStream; import java.io.File; import java.io.FileInputStream; import java.io.FileOutputStream; @@ -701,8 +700,7 @@ public void initTempDump(boolean isRestarting) { if (!file.exists()) { Files.createFile(file.toPath()); } - OutputStream rdfWriterTempTriplesOut = new FastBufferedOutputStream( - new FileOutputStream(file, isRestarting)); + OutputStream rdfWriterTempTriplesOut = new BufferedOutputStream(new FileOutputStream(file, isRestarting)); this.rdfWriterTempTriples = graph ? new NQuadsWriter(rdfWriterTempTriplesOut) : new NTriplesWriter(rdfWriterTempTriplesOut); this.rdfWriterTempTriples.startRDF(); diff --git a/qendpoint-store/src/main/java/com/the_qa_company/qendpoint/store/MergeRunnable.java b/qendpoint-store/src/main/java/com/the_qa_company/qendpoint/store/MergeRunnable.java index d45ded52b..30af58313 100644 --- a/qendpoint-store/src/main/java/com/the_qa_company/qendpoint/store/MergeRunnable.java +++ b/qendpoint-store/src/main/java/com/the_qa_company/qendpoint/store/MergeRunnable.java @@ -4,7 +4,7 @@ import com.the_qa_company.qendpoint.core.compact.bitmap.MultiLayerBitmapWrapper; import com.the_qa_company.qendpoint.core.dictionary.Dictionary; import com.the_qa_company.qendpoint.core.enums.TripleComponentOrder; -import it.unimi.dsi.fastutil.io.FastBufferedOutputStream; +import org.spf4j.io.BufferedOutputStream; import com.the_qa_company.qendpoint.store.exception.EndpointStoreException; import com.the_qa_company.qendpoint.utils.BitArrayDisk; import com.the_qa_company.qendpoint.utils.OverrideHDTOptions; @@ -31,7 +31,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.BufferedOutputStream; import java.io.File; import java.io.FileOutputStream; import java.io.IOException; @@ -926,7 +925,7 @@ private void createHDTDump(String rdfInput, String hdtOutput) throws IOException } private void writeTempFile(RepositoryConnection connection, String file, boolean graph) throws IOException { - try (OutputStream out = new FastBufferedOutputStream(new FileOutputStream(file))) { + try (OutputStream out = new BufferedOutputStream(new FileOutputStream(file))) { RDFWriter writer = Rio.createWriter( endpoint.getHdt().getDictionary().supportGraphs() ? RDFFormat.NQUADS : RDFFormat.NTRIPLES, out); RepositoryResult repositoryResult = connection.getStatements(null, null, null, false); diff --git a/qendpoint-store/src/main/java/com/the_qa_company/qendpoint/tools/QEPSearch.java b/qendpoint-store/src/main/java/com/the_qa_company/qendpoint/tools/QEPSearch.java index 737375b8f..f4fa975e1 100644 --- a/qendpoint-store/src/main/java/com/the_qa_company/qendpoint/tools/QEPSearch.java +++ b/qendpoint-store/src/main/java/com/the_qa_company/qendpoint/tools/QEPSearch.java @@ -17,7 +17,7 @@ import com.the_qa_company.qendpoint.core.options.HDTOptions; import com.the_qa_company.qendpoint.core.options.HDTOptionsKeys; import com.the_qa_company.qendpoint.core.rdf.parsers.RDFDeltaFileParser; -import it.unimi.dsi.fastutil.io.FastBufferedInputStream; +import org.spf4j.io.BufferedInputStream; import com.the_qa_company.qendpoint.core.tools.HDTVerify; import com.the_qa_company.qendpoint.core.triples.IteratorTripleString; import com.the_qa_company.qendpoint.core.triples.TripleString; @@ -49,7 +49,6 @@ import org.eclipse.rdf4j.sail.helpers.AbstractNotifyingSail; import org.eclipse.rdf4j.sail.nativerdf.NativeStore; -import java.io.BufferedInputStream; import java.io.BufferedReader; import java.io.BufferedWriter; import java.io.File; @@ -1029,7 +1028,7 @@ private void executeDeltaFile() throws IOException { HDTOptionsKeys.PARSER_DELTAFILE_NO_EXCEPTION, true); try (RDFDeltaFileParser.DeltaFileReader reader = new RDFDeltaFileParser.DeltaFileReader( - new FastBufferedInputStream(Files.newInputStream(file)), spec)) { + new BufferedInputStream(Files.newInputStream(file)), spec)) { console.printLine(console.color(5, 5, 1) + "files .. " + console.colorReset() + reader.getSize()); console.printLine(console.color(5, 5, 1) + "start .. " + console.colorReset() + reader.getStart()); diff --git a/qendpoint-store/src/main/java/com/the_qa_company/qendpoint/utils/BitArrayDisk.java b/qendpoint-store/src/main/java/com/the_qa_company/qendpoint/utils/BitArrayDisk.java index f79e432a1..013335d93 100644 --- a/qendpoint-store/src/main/java/com/the_qa_company/qendpoint/utils/BitArrayDisk.java +++ b/qendpoint-store/src/main/java/com/the_qa_company/qendpoint/utils/BitArrayDisk.java @@ -1,13 +1,12 @@ package com.the_qa_company.qendpoint.utils; -import it.unimi.dsi.fastutil.io.FastBufferedInputStream; +import org.spf4j.io.BufferedInputStream; import com.the_qa_company.qendpoint.store.exception.EndpointStoreException; import org.eclipse.rdf4j.common.io.NioFile; import com.the_qa_company.qendpoint.core.compact.bitmap.ModifiableBitmap; import com.the_qa_company.qendpoint.core.listener.ProgressListener; import com.the_qa_company.qendpoint.core.util.io.IOUtil; -import java.io.BufferedInputStream; import java.io.Closeable; import java.io.EOFException; import java.io.File; @@ -141,8 +140,7 @@ private void initWordsArray(long nbits) throws IOException { int lastNonZero = -1; // read previous values - try (InputStream is = new FastBufferedInputStream( - Files.newInputStream(this.output.getFile().toPath()))) { + try (InputStream is = new BufferedInputStream(Files.newInputStream(this.output.getFile().toPath()))) { // skip header is.skipNBytes(8); for (int i = 0; i < this.words.length; i++) { diff --git a/qendpoint-store/src/main/java/com/the_qa_company/qendpoint/utils/FormatUtils.java b/qendpoint-store/src/main/java/com/the_qa_company/qendpoint/utils/FormatUtils.java index 9102fee25..64e3a36ff 100644 --- a/qendpoint-store/src/main/java/com/the_qa_company/qendpoint/utils/FormatUtils.java +++ b/qendpoint-store/src/main/java/com/the_qa_company/qendpoint/utils/FormatUtils.java @@ -2,13 +2,12 @@ import com.the_qa_company.qendpoint.core.options.ControlInfo; import com.the_qa_company.qendpoint.core.options.ControlInformation; -import it.unimi.dsi.fastutil.io.FastBufferedInputStream; +import org.spf4j.io.BufferedInputStream; import org.eclipse.rdf4j.query.resultio.QueryResultFormat; import org.eclipse.rdf4j.query.resultio.TupleQueryResultWriterRegistry; import org.eclipse.rdf4j.rio.RDFFormat; import org.eclipse.rdf4j.rio.Rio; -import java.io.BufferedInputStream; import java.io.File; import java.io.IOException; import java.io.InputStream; @@ -88,7 +87,7 @@ public static Optional getRDFWriterFormat(String acceptHeader) { */ public static byte[] readCookie(Path file, int size) { try (InputStream stream = Files.newInputStream(file)) { - return (size > 0x1000 ? new FastBufferedInputStream(stream) : stream).readNBytes(size); + return (size > 0x1000 ? new BufferedInputStream(stream) : stream).readNBytes(size); } catch (IOException e) { return new byte[0]; } diff --git a/qendpoint-store/src/test/java/com/the_qa_company/qendpoint/store/MergeRestartTest.java b/qendpoint-store/src/test/java/com/the_qa_company/qendpoint/store/MergeRestartTest.java index c4f69590d..a9f010ac3 100644 --- a/qendpoint-store/src/test/java/com/the_qa_company/qendpoint/store/MergeRestartTest.java +++ b/qendpoint-store/src/test/java/com/the_qa_company/qendpoint/store/MergeRestartTest.java @@ -2,7 +2,7 @@ import com.the_qa_company.qendpoint.core.compact.bitmap.MultiLayerBitmapWrapper; import com.the_qa_company.qendpoint.core.enums.TripleComponentOrder; -import it.unimi.dsi.fastutil.io.FastBufferedOutputStream; +import org.spf4j.io.BufferedOutputStream; import com.the_qa_company.qendpoint.utils.BitArrayDisk; import org.apache.commons.io.FileUtils; import org.eclipse.rdf4j.common.iteration.CloseableIteration; @@ -37,7 +37,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.BufferedOutputStream; import java.io.Closeable; import java.io.File; import java.io.FileOutputStream; @@ -889,7 +888,7 @@ private void executeTestRemoveHDT(File out, SailRepository repo, int id, int cou connection.remove(stm); }); writeInfoCount(out, count); - try (OutputStream buff = new FastBufferedOutputStream( + try (OutputStream buff = new BufferedOutputStream( new FileOutputStream(out.getAbsolutePath() + ".delta", true))) { buff.write(("REM HDT " + id + " / " + count + "\n").getBytes(StandardCharsets.UTF_8)); } @@ -914,7 +913,7 @@ private void executeTestRemoveRDF(File out, SailRepository repo, int id, int cou connection.remove(stm); }); writeInfoCount(out, count); - try (OutputStream buff = new FastBufferedOutputStream( + try (OutputStream buff = new BufferedOutputStream( new FileOutputStream(out.getAbsolutePath() + ".delta", true))) { buff.write(("REM RDF " + id + " / " + count + "\n").getBytes(StandardCharsets.UTF_8)); } @@ -938,7 +937,7 @@ private void executeTestAddRDF(File out, SailRepository repo, int id, int count) connection.add(stm); }); writeInfoCount(out, count); - try (OutputStream buff = new FastBufferedOutputStream( + try (OutputStream buff = new BufferedOutputStream( new FileOutputStream(out.getAbsolutePath() + ".delta", true))) { buff.write(("ADD RDF " + id + " / " + count + "\n").getBytes(StandardCharsets.UTF_8)); } @@ -963,7 +962,7 @@ private void executeTestAddHDT(File out, SailRepository repo, int id, int count) connection.add(stm); }); writeInfoCount(out, count); - try (OutputStream buff = new FastBufferedOutputStream( + try (OutputStream buff = new BufferedOutputStream( new FileOutputStream(out.getAbsolutePath() + ".delta", true))) { buff.write(("ADD HDT " + id + " / " + count + "\n").getBytes(StandardCharsets.UTF_8)); } diff --git a/qendpoint-store/src/test/java/com/the_qa_company/qendpoint/store/Utility.java b/qendpoint-store/src/test/java/com/the_qa_company/qendpoint/store/Utility.java index be44a033c..f4252c408 100644 --- a/qendpoint-store/src/test/java/com/the_qa_company/qendpoint/store/Utility.java +++ b/qendpoint-store/src/test/java/com/the_qa_company/qendpoint/store/Utility.java @@ -1,6 +1,6 @@ package com.the_qa_company.qendpoint.store; -import it.unimi.dsi.fastutil.io.FastBufferedOutputStream; +import org.spf4j.io.BufferedOutputStream; import org.eclipse.rdf4j.model.IRI; import org.eclipse.rdf4j.model.Resource; import org.eclipse.rdf4j.model.Statement; @@ -38,7 +38,6 @@ import com.the_qa_company.qendpoint.core.hdt.HDTManager; import com.the_qa_company.qendpoint.core.options.HDTOptions; -import java.io.BufferedOutputStream; import java.io.File; import java.io.FileOutputStream; import java.io.IOException; @@ -161,7 +160,7 @@ public static Statement getFakeStatement(ValueFactory vf, int id) { private static void writeBigIndex(File file) throws IOException { ValueFactory vf = new MemValueFactory(); - try (OutputStream out = new FastBufferedOutputStream(new FileOutputStream(file))) { + try (OutputStream out = new BufferedOutputStream(new FileOutputStream(file))) { RDFWriter writer = Rio.createWriter(RDFFormat.NTRIPLES, out); writer.startRDF(); for (int i = 1; i <= COUNT; i++) { From d77479b338208c38386eb63a7cc043b8e1aadf30 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ha=CC=8Avard=20Ottestad?= Date: Wed, 19 Mar 2025 16:17:18 +0100 Subject: [PATCH 23/28] fix for shared TripleString, QuadString --- .../qendpoint/core/rdf/parsers/RDFParserRIOT.java | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/rdf/parsers/RDFParserRIOT.java b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/rdf/parsers/RDFParserRIOT.java index 963dff135..1278bc486 100644 --- a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/rdf/parsers/RDFParserRIOT.java +++ b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/rdf/parsers/RDFParserRIOT.java @@ -134,8 +134,6 @@ public void doParse(InputStream input, String baseUri, RDFNotation notation, boo } public static class ElemStringBuffer implements StreamRDF { - private final TripleString triple = new TripleString(); - private final QuadString quad = new QuadString(); private final RDFCallback callback; private final static AtomicInteger counter = new AtomicInteger(0); @@ -148,6 +146,8 @@ public ElemStringBuffer(RDFCallback callback) { @Override public void triple(Triple parsedTriple) { + TripleString triple = new TripleString(); + int i = counter.incrementAndGet(); if (i % 100 == 0 && i > 177271352) { System.out.println("Triple count: " + i); @@ -160,6 +160,8 @@ public void triple(Triple parsedTriple) { @Override public void quad(Quad parsedQuad) { + QuadString quad = new QuadString(); + int i = counter.incrementAndGet(); if (i % 100 == 0 && i > 177271352) { System.out.println("Quad count: " + i); From a2f995b18e834e0f867a12403f8ae35b0e668432 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ha=CC=8Avard=20Ottestad?= Date: Thu, 20 Mar 2025 08:23:44 +0100 Subject: [PATCH 24/28] cleanup --- .../utils/AsyncIteratorFetcherUnordered.java | 3 +- .../utils/PipedCopyIteratorUnordered.java | 19 +----- .../rdf/parsers/ConcurrentInputStream.java | 12 ---- .../core/rdf/parsers/RDFParserRIOT.java | 66 +++++++++---------- 4 files changed, 33 insertions(+), 67 deletions(-) diff --git a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/iterator/utils/AsyncIteratorFetcherUnordered.java b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/iterator/utils/AsyncIteratorFetcherUnordered.java index b946eac41..6d0e430f9 100644 --- a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/iterator/utils/AsyncIteratorFetcherUnordered.java +++ b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/iterator/utils/AsyncIteratorFetcherUnordered.java @@ -17,7 +17,6 @@ public class AsyncIteratorFetcherUnordered extends AsyncIteratorFetcher { public static final int BUFFER = 1024 * 4; private final Iterator iterator; - private final Lock lock = new ReentrantLock(); private boolean end; volatile Queue[] queue = new Queue[] { new ArrayDeque(BUFFER), new ArrayDeque(BUFFER), new ArrayDeque(BUFFER), new ArrayDeque(BUFFER), new ArrayDeque(BUFFER), new ArrayDeque(BUFFER), new ArrayDeque(BUFFER), @@ -80,7 +79,7 @@ public E get() { ArrayList objects = new ArrayList<>(BUFFER); for (int i = 0; i < BUFFER && iterator.hasNext(); i++) { - es.add(iterator.next()); + objects.add(iterator.next()); } es.addAll(objects); diff --git a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/iterator/utils/PipedCopyIteratorUnordered.java b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/iterator/utils/PipedCopyIteratorUnordered.java index a7760d692..bb9aeb12a 100644 --- a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/iterator/utils/PipedCopyIteratorUnordered.java +++ b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/iterator/utils/PipedCopyIteratorUnordered.java @@ -19,8 +19,6 @@ public class PipedCopyIteratorUnordered extends PipedCopyIterator { - private static final AtomicInteger counter = new AtomicInteger(0); - /** * RuntimeException generated by the PipedCopyIterator * @@ -170,7 +168,6 @@ public boolean hasNext() { } if (obj.end()) { - System.out.println("End of queue: " + counter.get()); end = true; if (exception != null) { throw exception; @@ -210,7 +207,7 @@ private QueueObject checkAllQueues(QueueObject originalObj) { obj = queueObjects.poll(); if (obj == null) { try { - Thread.sleep(100); + Thread.sleep(10); } catch (InterruptedException e) { throw new RuntimeException(e); } @@ -229,16 +226,6 @@ private QueueObject checkAllQueues(QueueObject originalObj) { return obj; } - private boolean isEnd() { - for (AtomicBoolean atomicBoolean : queueEnd) { - if (!atomicBoolean.get()) { - return false; - } - } - - return true; - } - private QueueObject iterateThroughAllQueues(QueueObject obj) throws InterruptedException { while (obj == null) { for (int i = 0; i < queue.length; i++) { @@ -295,10 +282,6 @@ public T next() { if (!hasNext()) { return null; } - int i = counter.incrementAndGet(); - if (i % 100 == 0 && i > 176451821) { - System.out.println("PipedCopyIteratorUnordered: " + i); - } T next = this.next; this.next = null; return next; diff --git a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/rdf/parsers/ConcurrentInputStream.java b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/rdf/parsers/ConcurrentInputStream.java index 03674807c..b5d0fb48d 100644 --- a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/rdf/parsers/ConcurrentInputStream.java +++ b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/rdf/parsers/ConcurrentInputStream.java @@ -59,19 +59,7 @@ private void startReadingThread() { try (BufferedReader reader = new BufferedReader(new InputStreamReader(source, StandardCharsets.UTF_8))) { String line; int currentStreamIndex = 0; - long lineCount = 0; - long start = System.currentTimeMillis(); while ((line = reader.readLine()) != null) { -// lineCount++; -// if (lineCount == 1000000) { -// long end = System.currentTimeMillis(); -// long duration = end - start; -// // print lines per second -// System.out.println(String.format("ConcurrentInputStream lines per second: %,d", -// ((int) Math.floor(lineCount / (duration / 1000.0))))); -// start = end; -// lineCount = 0; -// } byte[] data = (line + "\n").getBytes(StandardCharsets.UTF_8); diff --git a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/rdf/parsers/RDFParserRIOT.java b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/rdf/parsers/RDFParserRIOT.java index 1278bc486..d6e96d287 100644 --- a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/rdf/parsers/RDFParserRIOT.java +++ b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/rdf/parsers/RDFParserRIOT.java @@ -18,38 +18,48 @@ package com.the_qa_company.qendpoint.core.rdf.parsers; -import java.io.FileNotFoundException; -import java.io.InputStream; -import java.util.ArrayList; -import java.util.concurrent.atomic.AtomicInteger; - +import com.the_qa_company.qendpoint.core.enums.RDFNotation; +import com.the_qa_company.qendpoint.core.exceptions.NotImplementedException; +import com.the_qa_company.qendpoint.core.exceptions.ParserException; import com.the_qa_company.qendpoint.core.quad.QuadString; +import com.the_qa_company.qendpoint.core.rdf.RDFParserCallback; +import com.the_qa_company.qendpoint.core.triples.TripleString; +import com.the_qa_company.qendpoint.core.util.io.IOUtil; import org.apache.jena.graph.Triple; import org.apache.jena.riot.Lang; import org.apache.jena.riot.RDFParser; import org.apache.jena.riot.lang.LabelToNode; import org.apache.jena.riot.system.StreamRDF; import org.apache.jena.sparql.core.Quad; -import com.the_qa_company.qendpoint.core.enums.RDFNotation; -import com.the_qa_company.qendpoint.core.exceptions.NotImplementedException; -import com.the_qa_company.qendpoint.core.exceptions.ParserException; -import com.the_qa_company.qendpoint.core.rdf.RDFParserCallback; -import com.the_qa_company.qendpoint.core.triples.TripleString; -import com.the_qa_company.qendpoint.core.util.io.IOUtil; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.io.FileNotFoundException; +import java.io.InputStream; +import java.util.ArrayList; + /** * @author mario.arias */ public class RDFParserRIOT implements RDFParserCallback { private static final Logger log = LoggerFactory.getLogger(RDFParserRIOT.class); + private static final int CORES = Runtime.getRuntime().availableProcessors(); + private void parse(InputStream stream, String baseUri, Lang lang, boolean keepBNode, ElemStringBuffer buffer) { - Thread.dumpStack(); + + if (lang != Lang.NQUADS || lang != Lang.NTRIPLES) { + if (keepBNode) { + RDFParser.source(stream).base(baseUri).lang(lang).labelToNode(LabelToNode.createUseLabelAsGiven()) + .parse(buffer); + } else { + RDFParser.source(stream).base(baseUri).lang(lang).parse(buffer); + } + return; + } if (keepBNode) { - ConcurrentInputStream cs = new ConcurrentInputStream(stream, 11); + ConcurrentInputStream cs = new ConcurrentInputStream(stream, CORES - 1); InputStream bnodes = cs.getBnodeStream(); @@ -116,16 +126,16 @@ public void doParse(String fileName, String baseUri, RDFNotation notation, boole @Override public void doParse(InputStream input, String baseUri, RDFNotation notation, boolean keepBNode, - RDFCallback callback) throws ParserException { + RDFCallback callback) throws ParserException { try { switch (notation) { - case NTRIPLES -> parse(input, baseUri, Lang.NTRIPLES, keepBNode, new ElemStringBuffer(callback)); - case NQUAD -> parse(input, baseUri, Lang.NQUADS, keepBNode, new ElemStringBuffer(callback)); - case RDFXML -> parse(input, baseUri, Lang.RDFXML, keepBNode, new ElemStringBuffer(callback)); - case N3, TURTLE -> parse(input, baseUri, Lang.TURTLE, keepBNode, new ElemStringBuffer(callback)); - case TRIG -> parse(input, baseUri, Lang.TRIG, keepBNode, new ElemStringBuffer(callback)); - case TRIX -> parse(input, baseUri, Lang.TRIX, keepBNode, new ElemStringBuffer(callback)); - default -> throw new NotImplementedException("Parser not found for format " + notation); + case NTRIPLES -> parse(input, baseUri, Lang.NTRIPLES, keepBNode, new ElemStringBuffer(callback)); + case NQUAD -> parse(input, baseUri, Lang.NQUADS, keepBNode, new ElemStringBuffer(callback)); + case RDFXML -> parse(input, baseUri, Lang.RDFXML, keepBNode, new ElemStringBuffer(callback)); + case N3, TURTLE -> parse(input, baseUri, Lang.TURTLE, keepBNode, new ElemStringBuffer(callback)); + case TRIG -> parse(input, baseUri, Lang.TRIG, keepBNode, new ElemStringBuffer(callback)); + case TRIX -> parse(input, baseUri, Lang.TRIX, keepBNode, new ElemStringBuffer(callback)); + default -> throw new NotImplementedException("Parser not found for format " + notation); } } catch (Exception e) { log.error("Unexpected exception.", e); @@ -135,23 +145,14 @@ public void doParse(InputStream input, String baseUri, RDFNotation notation, boo public static class ElemStringBuffer implements StreamRDF { private final RDFCallback callback; - private final static AtomicInteger counter = new AtomicInteger(0); public ElemStringBuffer(RDFCallback callback) { this.callback = callback; - - Runtime.getRuntime().addShutdownHook( - new Thread(() -> { System.out.println("Total triples parsed: " + counter.get()); })); } @Override public void triple(Triple parsedTriple) { TripleString triple = new TripleString(); - - int i = counter.incrementAndGet(); - if (i % 100 == 0 && i > 177271352) { - System.out.println("Triple count: " + i); - } triple.setAll(JenaNodeFormatter.format(parsedTriple.getSubject()), JenaNodeFormatter.format(parsedTriple.getPredicate()), JenaNodeFormatter.format(parsedTriple.getObject())); @@ -161,11 +162,6 @@ public void triple(Triple parsedTriple) { @Override public void quad(Quad parsedQuad) { QuadString quad = new QuadString(); - - int i = counter.incrementAndGet(); - if (i % 100 == 0 && i > 177271352) { - System.out.println("Quad count: " + i); - } quad.setAll(JenaNodeFormatter.format(parsedQuad.getSubject()), JenaNodeFormatter.format(parsedQuad.getPredicate()), JenaNodeFormatter.format(parsedQuad.getObject()), JenaNodeFormatter.format(parsedQuad.getGraph())); From d8b8b3f02941bb6a0ef6c23ccf4cf0b215c7b2c8 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ha=CC=8Avard=20Ottestad?= Date: Thu, 20 Mar 2025 09:46:10 +0100 Subject: [PATCH 25/28] wip --- .../iterator/utils/AsyncIteratorFetcher.java | 7 ----- .../utils/AsyncIteratorFetcherUnordered.java | 27 +++++++------------ .../utils/PipedCopyIteratorUnordered.java | 22 +++++++-------- .../core/iterator/utils/SizeFetcher.java | 5 +--- .../rdf/parsers/ConcurrentInputStream.java | 3 +++ .../core/rdf/parsers/RDFParserRIOT.java | 21 ++++++++------- .../org/apache/jena/iri/impl/LexerFixer.java | 22 +++++++++++++++ 7 files changed, 57 insertions(+), 50 deletions(-) create mode 100644 qendpoint-core/src/main/java/org/apache/jena/iri/impl/LexerFixer.java diff --git a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/iterator/utils/AsyncIteratorFetcher.java b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/iterator/utils/AsyncIteratorFetcher.java index 03d67d3f6..8eb9e77eb 100644 --- a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/iterator/utils/AsyncIteratorFetcher.java +++ b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/iterator/utils/AsyncIteratorFetcher.java @@ -41,20 +41,16 @@ public AsyncIteratorFetcher(Iterator iterator) { // } // } - AtomicInteger counter = new AtomicInteger(0); - @Override public E get() { E poll = queue.poll(); if (poll != null) { - counter.incrementAndGet(); return poll; } synchronized (this) { poll = queue.poll(); if (poll != null) { - counter.incrementAndGet(); return poll; } @@ -68,12 +64,9 @@ public E get() { } this.queue = newqueue; if (poll != null) { - counter.incrementAndGet(); return poll; } - System.out.println("AsyncIteratorFetcher: " + counter.get()); - end = true; return null; } diff --git a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/iterator/utils/AsyncIteratorFetcherUnordered.java b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/iterator/utils/AsyncIteratorFetcherUnordered.java index 6d0e430f9..fa9930039 100644 --- a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/iterator/utils/AsyncIteratorFetcherUnordered.java +++ b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/iterator/utils/AsyncIteratorFetcherUnordered.java @@ -15,16 +15,19 @@ * @author Antoine Willerval */ public class AsyncIteratorFetcherUnordered extends AsyncIteratorFetcher { + + private static final int CORES = Runtime.getRuntime().availableProcessors(); + public static final int BUFFER = 1024 * 4; private final Iterator iterator; private boolean end; - volatile Queue[] queue = new Queue[] { new ArrayDeque(BUFFER), new ArrayDeque(BUFFER), new ArrayDeque(BUFFER), - new ArrayDeque(BUFFER), new ArrayDeque(BUFFER), new ArrayDeque(BUFFER), new ArrayDeque(BUFFER), - new ArrayDeque(BUFFER), new ArrayDeque(BUFFER), new ArrayDeque(BUFFER), new ArrayDeque(BUFFER), - new ArrayDeque(BUFFER), new ArrayDeque(BUFFER), new ArrayDeque(BUFFER), new ArrayDeque(BUFFER), - new ArrayDeque(BUFFER), }; + volatile Queue[] queue = new Queue[CORES * 2]; - AtomicInteger counter = new AtomicInteger(0); + { + for (int i = 0; i < queue.length; i++) { + queue[i] = new ArrayDeque<>(BUFFER); + } + } public AsyncIteratorFetcherUnordered(Iterator iterator) { super(iterator); @@ -47,7 +50,6 @@ public E get() { E poll = eQueue.poll(); if (poll != null) { - counter.incrementAndGet(); return poll; } } @@ -64,7 +66,6 @@ public E get() { E poll = es.poll(); if (poll != null) { - counter.incrementAndGet(); return poll; } @@ -76,13 +77,9 @@ public E get() { if (poll == null) { if (iterator.hasNext()) { poll = iterator.next(); - ArrayList objects = new ArrayList<>(BUFFER); - for (int i = 0; i < BUFFER && iterator.hasNext(); i++) { - objects.add(iterator.next()); + es.add(iterator.next()); } - - es.addAll(objects); } } @@ -90,7 +87,6 @@ public E get() { if (poll == null) { queue[index] = null; } else { - counter.incrementAndGet(); return poll; } } @@ -106,7 +102,6 @@ public E get() { E poll = eQueue.poll(); if (poll != null) { - counter.incrementAndGet(); return poll; } } @@ -117,12 +112,10 @@ public E get() { synchronized (this) { if (iterator.hasNext()) { E poll = iterator.next(); - counter.incrementAndGet(); return poll; } } - System.out.println("AsyncIteratorFetcherUnordered: " + counter.get()); end = true; return null; diff --git a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/iterator/utils/PipedCopyIteratorUnordered.java b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/iterator/utils/PipedCopyIteratorUnordered.java index bb9aeb12a..c8e426615 100644 --- a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/iterator/utils/PipedCopyIteratorUnordered.java +++ b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/iterator/utils/PipedCopyIteratorUnordered.java @@ -19,6 +19,8 @@ public class PipedCopyIteratorUnordered extends PipedCopyIterator { + private static final int CORES = Runtime.getRuntime().availableProcessors(); + /** * RuntimeException generated by the PipedCopyIterator * @@ -112,19 +114,13 @@ public T get() { } } - private final ArrayBlockingQueue>[] queue = new ArrayBlockingQueue[] { - new ArrayBlockingQueue<>(16 * 1024), new ArrayBlockingQueue<>(16 * 1024), - new ArrayBlockingQueue<>(16 * 1024), new ArrayBlockingQueue<>(16 * 1024), - new ArrayBlockingQueue<>(16 * 1024), new ArrayBlockingQueue<>(16 * 1024), - new ArrayBlockingQueue<>(16 * 1024), new ArrayBlockingQueue<>(16 * 1024), - new ArrayBlockingQueue<>(16 * 1024), new ArrayBlockingQueue<>(16 * 1024), - new ArrayBlockingQueue<>(16 * 1024), new ArrayBlockingQueue<>(16 * 1024), - new ArrayBlockingQueue<>(16 * 1024), new ArrayBlockingQueue<>(16 * 1024), - new ArrayBlockingQueue<>(16 * 1024), new ArrayBlockingQueue<>(16 * 1024), - new ArrayBlockingQueue<>(16 * 1024), new ArrayBlockingQueue<>(16 * 1024), - new ArrayBlockingQueue<>(16 * 1024), new ArrayBlockingQueue<>(16 * 1024), - new ArrayBlockingQueue<>(16 * 1024), new ArrayBlockingQueue<>(16 * 1024), - new ArrayBlockingQueue<>(16 * 1024), new ArrayBlockingQueue<>(16 * 1024), }; + private final ArrayBlockingQueue>[] queue = new ArrayBlockingQueue[CORES * 2]; + + { + for (int i = 0; i < queue.length; i++) { + queue[i] = new ArrayBlockingQueue<>(16 * 1024); + } + } private final AtomicBoolean[] queueEnd = new AtomicBoolean[queue.length]; diff --git a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/iterator/utils/SizeFetcher.java b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/iterator/utils/SizeFetcher.java index d775f5f83..ac9320cfe 100644 --- a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/iterator/utils/SizeFetcher.java +++ b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/iterator/utils/SizeFetcher.java @@ -30,7 +30,7 @@ public static SizeFetcher of(Supplier supplier, ToLongFunction size private final long maxSize; - private volatile long size; + private long size; public SizeFetcher(Supplier supplier, ToLongFunction sizeGetter, long maxSize) { this.supplier = supplier; @@ -50,9 +50,6 @@ public E get() { } size += sizeGetter.applyAsLong(e); -// if (size % 100000 == 0) { -// System.out.println("Size: " + size); -// } return e; } diff --git a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/rdf/parsers/ConcurrentInputStream.java b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/rdf/parsers/ConcurrentInputStream.java index b5d0fb48d..96b1bc894 100644 --- a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/rdf/parsers/ConcurrentInputStream.java +++ b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/rdf/parsers/ConcurrentInputStream.java @@ -60,6 +60,9 @@ private void startReadingThread() { String line; int currentStreamIndex = 0; while ((line = reader.readLine()) != null) { + if (line.isEmpty()) { + continue; // Skip empty lines + } byte[] data = (line + "\n").getBytes(StandardCharsets.UTF_8); diff --git a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/rdf/parsers/RDFParserRIOT.java b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/rdf/parsers/RDFParserRIOT.java index d6e96d287..ecac78e85 100644 --- a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/rdf/parsers/RDFParserRIOT.java +++ b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/rdf/parsers/RDFParserRIOT.java @@ -26,6 +26,7 @@ import com.the_qa_company.qendpoint.core.triples.TripleString; import com.the_qa_company.qendpoint.core.util.io.IOUtil; import org.apache.jena.graph.Triple; +import org.apache.jena.iri.impl.LexerFixer; import org.apache.jena.riot.Lang; import org.apache.jena.riot.RDFParser; import org.apache.jena.riot.lang.LabelToNode; @@ -48,7 +49,7 @@ public class RDFParserRIOT implements RDFParserCallback { private void parse(InputStream stream, String baseUri, Lang lang, boolean keepBNode, ElemStringBuffer buffer) { - if (lang != Lang.NQUADS || lang != Lang.NTRIPLES) { + if (lang != Lang.NQUADS && lang != Lang.NTRIPLES) { if (keepBNode) { RDFParser.source(stream).base(baseUri).lang(lang).labelToNode(LabelToNode.createUseLabelAsGiven()) .parse(buffer); @@ -59,6 +60,8 @@ private void parse(InputStream stream, String baseUri, Lang lang, boolean keepBN } if (keepBNode) { + LexerFixer.fixLexers(); + ConcurrentInputStream cs = new ConcurrentInputStream(stream, CORES - 1); InputStream bnodes = cs.getBnodeStream(); @@ -126,16 +129,16 @@ public void doParse(String fileName, String baseUri, RDFNotation notation, boole @Override public void doParse(InputStream input, String baseUri, RDFNotation notation, boolean keepBNode, - RDFCallback callback) throws ParserException { + RDFCallback callback) throws ParserException { try { switch (notation) { - case NTRIPLES -> parse(input, baseUri, Lang.NTRIPLES, keepBNode, new ElemStringBuffer(callback)); - case NQUAD -> parse(input, baseUri, Lang.NQUADS, keepBNode, new ElemStringBuffer(callback)); - case RDFXML -> parse(input, baseUri, Lang.RDFXML, keepBNode, new ElemStringBuffer(callback)); - case N3, TURTLE -> parse(input, baseUri, Lang.TURTLE, keepBNode, new ElemStringBuffer(callback)); - case TRIG -> parse(input, baseUri, Lang.TRIG, keepBNode, new ElemStringBuffer(callback)); - case TRIX -> parse(input, baseUri, Lang.TRIX, keepBNode, new ElemStringBuffer(callback)); - default -> throw new NotImplementedException("Parser not found for format " + notation); + case NTRIPLES -> parse(input, baseUri, Lang.NTRIPLES, keepBNode, new ElemStringBuffer(callback)); + case NQUAD -> parse(input, baseUri, Lang.NQUADS, keepBNode, new ElemStringBuffer(callback)); + case RDFXML -> parse(input, baseUri, Lang.RDFXML, keepBNode, new ElemStringBuffer(callback)); + case N3, TURTLE -> parse(input, baseUri, Lang.TURTLE, keepBNode, new ElemStringBuffer(callback)); + case TRIG -> parse(input, baseUri, Lang.TRIG, keepBNode, new ElemStringBuffer(callback)); + case TRIX -> parse(input, baseUri, Lang.TRIX, keepBNode, new ElemStringBuffer(callback)); + default -> throw new NotImplementedException("Parser not found for format " + notation); } } catch (Exception e) { log.error("Unexpected exception.", e); diff --git a/qendpoint-core/src/main/java/org/apache/jena/iri/impl/LexerFixer.java b/qendpoint-core/src/main/java/org/apache/jena/iri/impl/LexerFixer.java new file mode 100644 index 000000000..779313acb --- /dev/null +++ b/qendpoint-core/src/main/java/org/apache/jena/iri/impl/LexerFixer.java @@ -0,0 +1,22 @@ +package org.apache.jena.iri.impl; + +import java.io.Reader; + +public class LexerFixer { + + private static final int CORES = Runtime.getRuntime().availableProcessors(); + + public static void fixLexers() { + Parser.lexers = new Lexer[CORES * 4][]; + for (int i = 0; i < Parser.lexers.length; i++) { + Parser.lexers[i] = new Lexer[] { new LexerScheme((Reader) null), new LexerUserinfo((Reader) null), + new LexerHost((Reader) null), new LexerPort((Reader) null), new LexerPath((Reader) null), + new LexerQuery((Reader) null), new LexerFragment((Reader) null), new LexerXHost((Reader) null), }; + } + } + + public static void printLexerSize() { + int length = Parser.lexers.length; + System.out.println("Lexer size: " + length); + } +} From 0beedd3c27cb6b539fafb7553002588d5dfcddac Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ha=CC=8Avard=20Ottestad?= Date: Thu, 27 Mar 2025 09:15:18 +0100 Subject: [PATCH 26/28] minor optimisations --- .../rdf/parsers/ConcurrentInputStream.java | 53 +++++++++++-------- .../qendpoint/core/triples/TripleString.java | 2 +- 2 files changed, 33 insertions(+), 22 deletions(-) diff --git a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/rdf/parsers/ConcurrentInputStream.java b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/rdf/parsers/ConcurrentInputStream.java index 96b1bc894..39e4c96b7 100644 --- a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/rdf/parsers/ConcurrentInputStream.java +++ b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/rdf/parsers/ConcurrentInputStream.java @@ -1,5 +1,8 @@ package com.the_qa_company.qendpoint.core.rdf.parsers; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + import java.io.BufferedReader; import java.io.IOException; import java.io.InputStream; @@ -10,6 +13,7 @@ public class ConcurrentInputStream { + private static final Logger log = LoggerFactory.getLogger(ConcurrentInputStream.class); private final InputStream source; private final int numberOfStreams; @@ -55,7 +59,30 @@ private void setupPipes() { } private void startReadingThread() { - readerThread = new Thread(() -> { + readerThread = new Thread(new ReaderThread()); + + readerThread.setName("ConcurrentInputStream reader"); + readerThread.setDaemon(true); + readerThread.start(); + } + + /** + * Returns the stream for blank-node lines only. + */ + public InputStream getBnodeStream() { + return bnodeInputStream; + } + + /** + * Returns the array of InputStreams that share all concurrently read data. + */ + public InputStream[] getStreams() { + return pipedInputStreams; + } + + private class ReaderThread implements Runnable { + @Override + public void run() { try (BufferedReader reader = new BufferedReader(new InputStreamReader(source, StandardCharsets.UTF_8))) { String line; int currentStreamIndex = 0; @@ -77,6 +104,7 @@ private void startReadingThread() { } } } catch (IOException e) { + log.error("Error reading input stream", e); // If there's a read error, close everything. } finally { // Close all output streams to signal EOF @@ -89,27 +117,10 @@ private void startReadingThread() { try { bnodeOutputStream.close(); - } catch (IOException ignored) { + } catch (IOException e) { + log.error("Error closing bnodeOutputStream", e); } } - }); - - readerThread.setName("ConcurrentInputStream reader"); - readerThread.setDaemon(true); - readerThread.start(); - } - - /** - * Returns the stream for blank-node lines only. - */ - public InputStream getBnodeStream() { - return bnodeInputStream; - } - - /** - * Returns the array of InputStreams that share all concurrently read data. - */ - public InputStream[] getStreams() { - return pipedInputStreams; + } } } diff --git a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/triples/TripleString.java b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/triples/TripleString.java index d0abf9642..e9690f0dd 100644 --- a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/triples/TripleString.java +++ b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/triples/TripleString.java @@ -422,7 +422,7 @@ public String toString() { * @throws IOException when IOException occurs */ public CharSequence asNtriple() throws IOException { - StringBuilder str = new StringBuilder(); + StringBuilder str = new StringBuilder(subject.length() + predicate.length() + object.length() + 10); this.dumpNtriple(str); return str; } From 5b37f171edb9fa2c5c91006e06d1bb10f2cef719 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ha=CC=8Avard=20Ottestad?= Date: Thu, 27 Mar 2025 14:37:32 +0100 Subject: [PATCH 27/28] added a TurtleChunker --- .../core/rdf/parsers/TurtleChunker.java | 386 ++++++++++++++++++ 1 file changed, 386 insertions(+) create mode 100644 qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/rdf/parsers/TurtleChunker.java diff --git a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/rdf/parsers/TurtleChunker.java b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/rdf/parsers/TurtleChunker.java new file mode 100644 index 000000000..3779c341a --- /dev/null +++ b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/rdf/parsers/TurtleChunker.java @@ -0,0 +1,386 @@ +package com.the_qa_company.qendpoint.core.rdf.parsers; + +import java.io.FileNotFoundException; +import java.io.FileReader; +import java.io.IOException; +import java.io.Reader; +import java.lang.invoke.MethodHandle; +import java.lang.invoke.MethodHandles; +import java.lang.invoke.MethodType; +import java.util.ArrayDeque; +import java.util.Deque; +import java.util.Iterator; +import java.util.NoSuchElementException; + +/** + * A preprocessor for Turtle files, which reads the file and returns an iterator + * of chunks where each chunk is a block of turtle data with a . termination. + * This might be a single statement, or multiple statements. + */ +public class TurtleChunker { + + private enum State { + DEFAULT, IRI, LITERAL, MULTILINE_LITERAL + } + + private State state = State.DEFAULT; + + private static final int BUFFER_SIZE = 1024 * 1024 * 4; + private final Reader reader; + private final char[] chunkBuf = new char[BUFFER_SIZE]; + private int bufPos = 0, bufLen = 0; + + private final StringBuilder tokenBuffer = new StringBuilder(); + + private final Deque nestingStack = new ArrayDeque<>(); + + private char literalDelimiter; + + private final MethodHandle[] defaultActions = new MethodHandle[256]; + + public TurtleChunker(Reader reader) { + this.reader = reader; + buildDefaultActions(); + } + + private void buildDefaultActions() { + try { + MethodHandles.Lookup lookup = MethodHandles.lookup(); + + MethodType rawMt = MethodType.methodType(void.class, char.class); + + MethodHandle rawLt = lookup.findVirtual(TurtleChunker.class, "handleLtInDefault", rawMt); + MethodHandle rawHash = lookup.findVirtual(TurtleChunker.class, "handleHashInDefault", rawMt); + + MethodHandle rawLParen = lookup.findVirtual(TurtleChunker.class, "handleLParenInDefault", rawMt); + MethodHandle rawRParen = lookup.findVirtual(TurtleChunker.class, "handleRParenInDefault", rawMt); + MethodHandle rawLBrack = lookup.findVirtual(TurtleChunker.class, "handleLBrackInDefault", rawMt); + MethodHandle rawRBrack = lookup.findVirtual(TurtleChunker.class, "handleRBrackInDefault", rawMt); + MethodHandle rawQ1 = lookup.findVirtual(TurtleChunker.class, "handleQuote1InDefault", rawMt); + MethodHandle rawQ2 = lookup.findVirtual(TurtleChunker.class, "handleQuote2InDefault", rawMt); + MethodHandle rawDot = lookup.findVirtual(TurtleChunker.class, "handleDotInDefault", rawMt); + + // Bind each method to `this` so it becomes (char)->void + MethodHandle boundLt = rawLt.bindTo(this); + MethodHandle boundHash = rawHash.bindTo(this); + + MethodHandle boundLParen = rawLParen.bindTo(this); + MethodHandle boundRParen = rawRParen.bindTo(this); + MethodHandle boundLBrack = rawLBrack.bindTo(this); + MethodHandle boundRBrack = rawRBrack.bindTo(this); + MethodHandle boundQ1 = rawQ1.bindTo(this); + MethodHandle boundQ2 = rawQ2.bindTo(this); + MethodHandle boundDot = rawDot.bindTo(this); + + defaultActions['<'] = boundLt; + defaultActions['#'] = boundHash; + defaultActions['('] = boundLParen; + defaultActions[')'] = boundRParen; + defaultActions['['] = boundLBrack; + defaultActions[']'] = boundRBrack; + defaultActions['\''] = boundQ1; + defaultActions['"'] = boundQ2; + defaultActions['.'] = boundDot; + + } catch (NoSuchMethodException | IllegalAccessException e) { + throw new RuntimeException("Failed to build defaultActions", e); + } + } + + private String parseNextBlock() throws IOException { + while (true) { + refillIfNeeded(); + if (bufLen == 0) { + // no more data => produce any leftover partial + if (!tokenBuffer.isEmpty()) { + // Possibly return a final partial statement + String leftover = tokenBuffer.toString().trim(); + tokenBuffer.setLength(0); + return leftover.isEmpty() ? null : leftover; + } + return null; // truly no more + } + + switch (state) { + case DEFAULT -> parseDefaultOneStep(); + case IRI -> parseIriOneStep(); + case LITERAL -> parseLiteralOneStep(); + case MULTILINE_LITERAL -> parseMultilineLiteralOneStep(); + } + + // Check if we completed a "block"? + // The condition: parseDefault encountered '.' outside nesting => + // calls completeToken() + if (finishedOneBlock != null) { + String block = finishedOneBlock; + finishedOneBlock = null; + return block; + } + } + } + + private String finishedOneBlock = null; + + private void parseDefaultOneStep() { + char ch = chunkBuf[bufPos++]; + tokenBuffer.append(ch); + + MethodHandle mh = defaultActions[ch & 0xFF]; + if (mh != null) { + try { + mh.invokeExact(ch); // (char)->void + } catch (Throwable t) { + throw new RuntimeException(t); + } + } + + } + + private void parseIriOneStep() { + char ch = chunkBuf[bufPos]; + tokenBuffer.append(ch); + bufPos++; + if (ch == '>') { + state = State.DEFAULT; + } + + } + + private void parseLiteralOneStep() { + char ch = chunkBuf[bufPos++]; + tokenBuffer.append(ch); + + if (ch == literalDelimiter && !isEscaped()) { + state = State.DEFAULT; + } + } + + private void parseMultilineLiteralOneStep() { + char ch = chunkBuf[bufPos++]; + tokenBuffer.append(ch); + + if (ch == literalDelimiter && !isEscaped()) { + if (maybeEndTripleQuote()) { + state = State.DEFAULT; + } + } + } + + private void handleLtInDefault(char ch) { + state = State.IRI; + } + + private void handleHashInDefault(char ch) { + skipComment(); + } + + private void handleLParenInDefault(char ch) { + nestingStack.push('('); + } + + private void handleRParenInDefault(char ch) { + if (!nestingStack.isEmpty()) { + nestingStack.pop(); + } + } + + private void handleLBrackInDefault(char ch) { + nestingStack.push('['); + } + + private void handleRBrackInDefault(char ch) { + if (!nestingStack.isEmpty()) { + nestingStack.pop(); + } + } + + private void handleQuote1InDefault(char ch) { + if (checkForTripleQuote(ch)) { + state = State.MULTILINE_LITERAL; + literalDelimiter = ch; + } else { + state = State.LITERAL; + literalDelimiter = ch; + } + } + + private void handleQuote2InDefault(char ch) { + if (checkForTripleQuote(ch)) { + state = State.MULTILINE_LITERAL; + literalDelimiter = ch; + } else { + state = State.LITERAL; + literalDelimiter = ch; + } + } + + private void handleDotInDefault(char ch) { + if (nestingStack.isEmpty()) { + String block = tokenBuffer.toString(); + if (tokenBuffer.capacity() > 512 * 1024) { + tokenBuffer.setLength(0); + tokenBuffer.trimToSize(); + } else { + tokenBuffer.setLength(0); + } + if (!block.isEmpty()) { + finishedOneBlock = block; + } + } + } + + private void skipComment() { + while (true) { + if (bufPos >= bufLen) { + return; + } + char ch = chunkBuf[bufPos++]; + tokenBuffer.append(ch); + if (ch == '\n') { + return; + } + } + } + + private boolean checkForTripleQuote(char quoteChar) { + if (bufPos + 1 < bufLen) { + if (chunkBuf[bufPos] == quoteChar && chunkBuf[bufPos + 1] == quoteChar) { + tokenBuffer.append(chunkBuf[bufPos++]); + tokenBuffer.append(chunkBuf[bufPos++]); + return true; + } + } + return false; + } + + private boolean maybeEndTripleQuote() { + if (bufPos + 1 < bufLen) { + if (chunkBuf[bufPos] == literalDelimiter && chunkBuf[bufPos + 1] == literalDelimiter) { + tokenBuffer.append(chunkBuf[bufPos++]); + tokenBuffer.append(chunkBuf[bufPos++]); + return true; + } + } + return false; + } + + private boolean isEscaped() { + int len = tokenBuffer.length(); + int backslashCount = 0; + for (int i = len - 2; i >= 0; i--) { + if (tokenBuffer.charAt(i) == '\\') { + backslashCount++; + } else { + break; + } + } + return (backslashCount % 2 == 1); + } + + private void refillIfNeeded() throws IOException { + while (bufPos >= bufLen) { + bufLen = reader.read(chunkBuf); + bufPos = 0; + if (bufLen == -1) { + bufLen = 0; // EOF + return; + } + } + } + + public BlockIterator blockIterator() { + return new BlockIterator(); + } + + public class BlockIterator implements Iterator { + private String nextBlock; + private boolean done; + + String getPrefixes() { + StringBuilder sb = new StringBuilder(); + while (hasNext()) { + String lowerCase = nextBlock.trim().toLowerCase(); + if (lowerCase.isEmpty() || lowerCase.startsWith("#")) { + nextBlock = null; + } else if (lowerCase.startsWith("@prefix") || lowerCase.startsWith("@base")) { + sb.append(nextBlock.trim()).append("\n"); + nextBlock = null; + } else { + break; + } + } + return sb.toString(); + } + + @Override + public boolean hasNext() { + if (done) { + return false; + } + if (nextBlock != null) { + return true; + } + try { + nextBlock = parseNextBlock(); + if (nextBlock == null) { + done = true; + return false; + } + return true; + } catch (IOException e) { + done = true; + throw new RuntimeException("IO error during iteration", e); + } + } + + @Override + public String next() { + if (!hasNext()) { + throw new NoSuchElementException("No more blocks"); + } + String result = nextBlock; + nextBlock = null; + return result; + } + } + + public static void main(String[] args) throws FileNotFoundException { + + long actualStart = System.currentTimeMillis(); + + long start = System.currentTimeMillis(); + long count = 0; + + try (Reader sr = (new FileReader("/Users/havardottestad/Downloads/aria2c/latest-dump.ttl"))) { + TurtleChunker tokenizer = new TurtleChunker(sr); + BlockIterator it = tokenizer.blockIterator(); + + String prefixes = it.getPrefixes(); + System.out.println("Prefixes:\n" + prefixes); + + while (it.hasNext()) { + String block = it.next(); + count += block.trim().split("\n").length; + if (count > 10000000) { + System.out.println(count + " lines parsed"); + System.out.println(block); + // count per second with thousands separator + System.out.printf("Lines per second: %,d \n", count * 1000 / (System.currentTimeMillis() - start)); + start = System.currentTimeMillis(); + count = 0; + } + + } + } catch (IOException e) { + e.printStackTrace(); + } finally { + long actualEnd = System.currentTimeMillis(); + long total = actualEnd - actualStart; + + long minutes = total / 60000; + long seconds = (total % 60000) / 1000; + System.out.printf("Total time: %d:%02d%n", minutes, seconds); + + } + } +} From 1788cc01e4e8884d7b5c5e697c794a3538435b53 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ha=CC=8Avard=20Ottestad?= Date: Thu, 3 Apr 2025 16:19:06 +0200 Subject: [PATCH 28/28] Optimise TurtleChunker --- .../core/rdf/parsers/TurtleChunker.java | 370 ++++++++++++------ 1 file changed, 243 insertions(+), 127 deletions(-) diff --git a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/rdf/parsers/TurtleChunker.java b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/rdf/parsers/TurtleChunker.java index 3779c341a..3c40c192a 100644 --- a/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/rdf/parsers/TurtleChunker.java +++ b/qendpoint-core/src/main/java/com/the_qa_company/qendpoint/core/rdf/parsers/TurtleChunker.java @@ -1,12 +1,13 @@ package com.the_qa_company.qendpoint.core.rdf.parsers; -import java.io.FileNotFoundException; -import java.io.FileReader; +import java.io.ByteArrayOutputStream; +import java.io.FileInputStream; import java.io.IOException; -import java.io.Reader; +import java.io.InputStream; import java.lang.invoke.MethodHandle; import java.lang.invoke.MethodHandles; import java.lang.invoke.MethodType; +import java.nio.charset.StandardCharsets; import java.util.ArrayDeque; import java.util.Deque; import java.util.Iterator; @@ -19,6 +20,8 @@ */ public class TurtleChunker { + private int consecutiveBackslashes; + private enum State { DEFAULT, IRI, LITERAL, MULTILINE_LITERAL } @@ -26,32 +29,46 @@ private enum State { private State state = State.DEFAULT; private static final int BUFFER_SIZE = 1024 * 1024 * 4; - private final Reader reader; - private final char[] chunkBuf = new char[BUFFER_SIZE]; + + private final InputStream in; // CHANGED (was Reader) + private final byte[] chunkBuf = new byte[BUFFER_SIZE]; private int bufPos = 0, bufLen = 0; - private final StringBuilder tokenBuffer = new StringBuilder(); + /** + * Stores partial bytes if the block spans multiple reads. If we never + * refill mid-block, we won't need this. + */ + private final ByteArrayOutputStream partialBytes = new ByteArrayOutputStream(); // CHANGED private final Deque nestingStack = new ArrayDeque<>(); - - private char literalDelimiter; + private byte literalDelimiter; private final MethodHandle[] defaultActions = new MethodHandle[256]; + private String finishedOneBlock = null; - public TurtleChunker(Reader reader) { - this.reader = reader; + /** + * Indicates whether the current block has already crossed multiple reads + * (thus is partially in `partialBytes`). + */ + private boolean multiReadBlock = false; // CHANGED + /** + * Marks where the current block started in `chunkBuf` if not in multi-read + * mode. + */ + private int chunkStart = 0; // CHANGED + + public TurtleChunker(InputStream in) { // CHANGED + this.in = in; buildDefaultActions(); } private void buildDefaultActions() { try { MethodHandles.Lookup lookup = MethodHandles.lookup(); - - MethodType rawMt = MethodType.methodType(void.class, char.class); + MethodType rawMt = MethodType.methodType(void.class, byte.class); MethodHandle rawLt = lookup.findVirtual(TurtleChunker.class, "handleLtInDefault", rawMt); MethodHandle rawHash = lookup.findVirtual(TurtleChunker.class, "handleHashInDefault", rawMt); - MethodHandle rawLParen = lookup.findVirtual(TurtleChunker.class, "handleLParenInDefault", rawMt); MethodHandle rawRParen = lookup.findVirtual(TurtleChunker.class, "handleRParenInDefault", rawMt); MethodHandle rawLBrack = lookup.findVirtual(TurtleChunker.class, "handleLBrackInDefault", rawMt); @@ -60,10 +77,8 @@ private void buildDefaultActions() { MethodHandle rawQ2 = lookup.findVirtual(TurtleChunker.class, "handleQuote2InDefault", rawMt); MethodHandle rawDot = lookup.findVirtual(TurtleChunker.class, "handleDotInDefault", rawMt); - // Bind each method to `this` so it becomes (char)->void MethodHandle boundLt = rawLt.bindTo(this); MethodHandle boundHash = rawHash.bindTo(this); - MethodHandle boundLParen = rawLParen.bindTo(this); MethodHandle boundRParen = rawRParen.bindTo(this); MethodHandle boundLBrack = rawLBrack.bindTo(this); @@ -81,36 +96,40 @@ private void buildDefaultActions() { defaultActions['\''] = boundQ1; defaultActions['"'] = boundQ2; defaultActions['.'] = boundDot; - } catch (NoSuchMethodException | IllegalAccessException e) { throw new RuntimeException("Failed to build defaultActions", e); } } + /* + * ---------------------------------------------------------------- The main + * loop that reads & parses blocks. + * ---------------------------------------------------------------- + */ private String parseNextBlock() throws IOException { while (true) { - refillIfNeeded(); + if (bufPos >= bufLen) { + readMoreData(); + } if (bufLen == 0) { - // no more data => produce any leftover partial - if (!tokenBuffer.isEmpty()) { - // Possibly return a final partial statement - String leftover = tokenBuffer.toString().trim(); - tokenBuffer.setLength(0); - return leftover.isEmpty() ? null : leftover; + // no more data => produce leftover partial if any + if (partialBytes.size() > 0) { // CHANGED + partialBytes.reset(); // CHANGED + // CHANGED + String leftoverStr = partialBytes.toString(StandardCharsets.UTF_8); + leftoverStr = leftoverStr.trim(); + return leftoverStr.isEmpty() ? null : leftoverStr; } return null; // truly no more } switch (state) { - case DEFAULT -> parseDefaultOneStep(); - case IRI -> parseIriOneStep(); - case LITERAL -> parseLiteralOneStep(); - case MULTILINE_LITERAL -> parseMultilineLiteralOneStep(); + case DEFAULT -> parseDefaultOneStep(); + case IRI -> parseIriOneStep(); + case LITERAL -> parseLiteralOneStep(); + case MULTILINE_LITERAL -> parseMultilineLiteralOneStep(); } - // Check if we completed a "block"? - // The condition: parseDefault encountered '.' outside nesting => - // calls completeToken() if (finishedOneBlock != null) { String block = finishedOneBlock; finishedOneBlock = null; @@ -119,175 +138,263 @@ private String parseNextBlock() throws IOException { } } - private String finishedOneBlock = null; - - private void parseDefaultOneStep() { - char ch = chunkBuf[bufPos++]; - tokenBuffer.append(ch); - - MethodHandle mh = defaultActions[ch & 0xFF]; + /* + * ---------------------------------------------------------------- + * parseXxxOneStep methods: We do not append to partialBytes here unless we + * are finalizing the block. We parse in-place from chunkBuf for ASCII + * triggers, etc. + * ---------------------------------------------------------------- + */ + + private void parseDefaultOneStep() throws IOException { + byte b = nextByte(); + MethodHandle mh = defaultActions[b & 0xFF]; if (mh != null) { try { - mh.invokeExact(ch); // (char)->void + mh.invokeExact(b); // (byte)->void } catch (Throwable t) { + if (t instanceof IOException ioException) { + throw ioException; + } else if (t instanceof Error error) { + throw error; + } else if (t instanceof RuntimeException runtimeException) { + throw runtimeException; + } else if (t instanceof InterruptedException) { + Thread.currentThread().interrupt(); + throw new RuntimeException(t); + } throw new RuntimeException(t); } } - } private void parseIriOneStep() { - char ch = chunkBuf[bufPos]; - tokenBuffer.append(ch); - bufPos++; - if (ch == '>') { - state = State.DEFAULT; + while (bufPos < bufLen) { + byte b = nextByte(); + if (b == '>') { + state = State.DEFAULT; + return; + } } - } private void parseLiteralOneStep() { - char ch = chunkBuf[bufPos++]; - tokenBuffer.append(ch); + while (bufPos < bufLen) { + + byte b = nextByte(); + + if (b == '\\') { + consecutiveBackslashes++; + continue; + } + + boolean escaped = (consecutiveBackslashes % 2 == 1); + consecutiveBackslashes = 0; // reset whenever we see a non-backslash - if (ch == literalDelimiter && !isEscaped()) { - state = State.DEFAULT; + if (b == literalDelimiter && !escaped) { + state = State.DEFAULT; + return; + } } } - private void parseMultilineLiteralOneStep() { - char ch = chunkBuf[bufPos++]; - tokenBuffer.append(ch); + private void parseMultilineLiteralOneStep() throws IOException { - if (ch == literalDelimiter && !isEscaped()) { - if (maybeEndTripleQuote()) { - state = State.DEFAULT; + while (bufPos < bufLen) { + + byte b = nextByte(); + + if (b == '\\') { + consecutiveBackslashes++; + continue; + } + + boolean escaped = (consecutiveBackslashes % 2 == 1); + consecutiveBackslashes = 0; // reset whenever we see a non-backslash + + if (b == literalDelimiter && !escaped) { + if (checkForTripleQuote(literalDelimiter)) { + state = State.DEFAULT; + return; + } } } } - private void handleLtInDefault(char ch) { + /* + * ---------------------------------------------------------------- Special + * char handlers in DEFAULT state + * ---------------------------------------------------------------- + */ + + private void handleLtInDefault(byte b) { state = State.IRI; } - private void handleHashInDefault(char ch) { + private void handleHashInDefault(byte b) { skipComment(); } - private void handleLParenInDefault(char ch) { + private void handleLParenInDefault(byte b) { nestingStack.push('('); } - private void handleRParenInDefault(char ch) { + private void handleRParenInDefault(byte b) { if (!nestingStack.isEmpty()) { nestingStack.pop(); } } - private void handleLBrackInDefault(char ch) { + private void handleLBrackInDefault(byte b) { nestingStack.push('['); } - private void handleRBrackInDefault(char ch) { + private void handleRBrackInDefault(byte b) { if (!nestingStack.isEmpty()) { nestingStack.pop(); } } - private void handleQuote1InDefault(char ch) { - if (checkForTripleQuote(ch)) { + private void handleQuote1InDefault(byte b) throws IOException { + if (checkForTripleQuote(b)) { state = State.MULTILINE_LITERAL; - literalDelimiter = ch; + literalDelimiter = b; } else { state = State.LITERAL; - literalDelimiter = ch; + literalDelimiter = b; } } - private void handleQuote2InDefault(char ch) { - if (checkForTripleQuote(ch)) { + private void handleQuote2InDefault(byte b) throws IOException { + if (checkForTripleQuote(b)) { state = State.MULTILINE_LITERAL; - literalDelimiter = ch; + literalDelimiter = b; } else { state = State.LITERAL; - literalDelimiter = ch; + literalDelimiter = b; } } - private void handleDotInDefault(char ch) { + private void handleDotInDefault(byte b) { if (nestingStack.isEmpty()) { - String block = tokenBuffer.toString(); - if (tokenBuffer.capacity() > 512 * 1024) { - tokenBuffer.setLength(0); - tokenBuffer.trimToSize(); - } else { - tokenBuffer.setLength(0); + finalizeBlock(); + } + } + + /* + * ---------------------------------------------------------------- + * finalizeBlock: build the final statement string + * ---------------------------------------------------------------- + */ + private void finalizeBlock() { + if (!multiReadBlock) { + // The entire block is in chunkBuf from chunkStart..bufPos + int length = bufPos - chunkStart; + if (length <= 0) { + return; // nothing } - if (!block.isEmpty()) { - finishedOneBlock = block; + String block = new String(chunkBuf, chunkStart, length, StandardCharsets.UTF_8); + + chunkStart = bufPos; // next block starts here + finishedOneBlock = block; + } else { + // partial data is in partialBytes + leftover in chunkBuf + if (bufPos > chunkStart) { + partialBytes.write(chunkBuf, chunkStart, (bufPos - chunkStart)); // CHANGED } + String block = partialBytes.toString(StandardCharsets.UTF_8); + + partialBytes.reset(); // CHANGED + finishedOneBlock = block; + multiReadBlock = false; + chunkStart = bufPos; } } + /* + * ---------------------------------------------------------------- + * skipComment, tripleQuote, escaping checks We parse in place for + * detection. + * ---------------------------------------------------------------- + */ + private void skipComment() { while (true) { if (bufPos >= bufLen) { return; } - char ch = chunkBuf[bufPos++]; - tokenBuffer.append(ch); - if (ch == '\n') { + byte b = nextByte(); + // check if the byte represents an ASCII character, if not then it's + // not relevant to check + if ((b & 0x80) != 0) { + continue; + } + + if (b == '\n') { return; } } } - private boolean checkForTripleQuote(char quoteChar) { - if (bufPos + 1 < bufLen) { - if (chunkBuf[bufPos] == quoteChar && chunkBuf[bufPos + 1] == quoteChar) { - tokenBuffer.append(chunkBuf[bufPos++]); - tokenBuffer.append(chunkBuf[bufPos++]); - return true; - } - } - return false; + private byte nextByte() { + return chunkBuf[bufPos++]; } - private boolean maybeEndTripleQuote() { - if (bufPos + 1 < bufLen) { - if (chunkBuf[bufPos] == literalDelimiter && chunkBuf[bufPos + 1] == literalDelimiter) { - tokenBuffer.append(chunkBuf[bufPos++]); - tokenBuffer.append(chunkBuf[bufPos++]); - return true; - } + private boolean checkForTripleQuote(byte quoteChar) throws IOException { + if (bufPos >= bufLen) { + readMoreData(); + } + + if (bufPos >= bufLen) { + return false; } - return false; - } - private boolean isEscaped() { - int len = tokenBuffer.length(); - int backslashCount = 0; - for (int i = len - 2; i >= 0; i--) { - if (tokenBuffer.charAt(i) == '\\') { - backslashCount++; + if (chunkBuf[bufPos] == quoteChar) { + bufPos++; + if (bufPos >= bufLen) { + readMoreData(); + } + if (bufPos < bufLen) { + if (chunkBuf[bufPos] == quoteChar) { + bufPos++; + return true; + } + return false; } else { - break; + return false; } + + } else { + return false; } - return (backslashCount % 2 == 1); } - private void refillIfNeeded() throws IOException { - while (bufPos >= bufLen) { - bufLen = reader.read(chunkBuf); - bufPos = 0; - if (bufLen == -1) { - bufLen = 0; // EOF - return; - } + /* + * ---------------------------------------------------------------- + * readMoreData: if we run out of data & haven't ended the block, copy + * leftover from chunkBuf to partialBytes to avoid overwriting it. + * ---------------------------------------------------------------- + */ + private void readMoreData() throws IOException { + // If we haven't finished the current block + if (chunkStart < bufLen) { + partialBytes.write(chunkBuf, chunkStart, bufLen - chunkStart); // CHANGED + multiReadBlock = true; + } + chunkStart = 0; + bufLen = in.read(chunkBuf); + bufPos = 0; + if (bufLen == -1) { + bufLen = 0; // EOF } } + /* + * ---------------------------------------------------------------- + * BlockIterator + * ---------------------------------------------------------------- + */ + public BlockIterator blockIterator() { return new BlockIterator(); } @@ -344,43 +451,52 @@ public String next() { } } - public static void main(String[] args) throws FileNotFoundException { - + // -- Example main for testing -- + public static void main(String[] args) { + String filePath = "/Users/havardottestad/Documents/Programming/qEndpoint3/indexing/latest-dump.ttl"; // Update + // path long actualStart = System.currentTimeMillis(); - long start = System.currentTimeMillis(); long count = 0; + long total = 0; - try (Reader sr = (new FileReader("/Users/havardottestad/Downloads/aria2c/latest-dump.ttl"))) { + try (InputStream sr = new FileInputStream( + "/Users/havardottestad/Documents/Programming/qEndpoint3/indexing/latest-dump.ttl")) { TurtleChunker tokenizer = new TurtleChunker(sr); BlockIterator it = tokenizer.blockIterator(); + System.out.println("Processing with NIO AsynchronousFileChannel (blocking wait)..."); + String prefixes = it.getPrefixes(); System.out.println("Prefixes:\n" + prefixes); while (it.hasNext()) { String block = it.next(); - count += block.trim().split("\n").length; - if (count > 10000000) { + int length = block.trim().split("\n").length; + count += length; + total += length; + if (count > 10_000_000) { System.out.println(count + " lines parsed"); System.out.println(block); - // count per second with thousands separator System.out.printf("Lines per second: %,d \n", count * 1000 / (System.currentTimeMillis() - start)); + + System.out.printf("Lines per second (total): %,d \n", + total * 1000 / (System.currentTimeMillis() - actualStart)); + start = System.currentTimeMillis(); count = 0; } - } } catch (IOException e) { e.printStackTrace(); } finally { long actualEnd = System.currentTimeMillis(); - long total = actualEnd - actualStart; + long total2 = actualEnd - actualStart; - long minutes = total / 60000; - long seconds = (total % 60000) / 1000; + long minutes = total2 / 60000; + long seconds = (total2 % 60000) / 1000; + System.out.printf("Total: %,d \n", total); System.out.printf("Total time: %d:%02d%n", minutes, seconds); - } } }