You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@lucene.apache.org by dweiss <gi...@git.apache.org> on 2018/08/06 10:38:24 UTC
[GitHub] lucene-solr pull request #432: LUCENE-8438: RAMDirectory speed improvements ...
GitHub user dweiss opened a pull request:
https://github.com/apache/lucene-solr/pull/432
LUCENE-8438: RAMDirectory speed improvements and cleanup
PR for the issue.
You can merge this pull request into a Git repository by running:
$ git pull https://github.com/dweiss/lucene-solr LUCENE-8438
Alternatively you can review and apply these changes as the patch at:
https://github.com/apache/lucene-solr/pull/432.patch
To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:
This closes #432
----
commit 58cae6b35e19fc1f76d5cdcd27ecf61664421da2
Author: Dawid Weiss <dw...@...>
Date: 2018-08-03T09:55:35Z
Initial addition of ByteBuffers (input, output, directory) and ByteArrayIndexInput that conforms to the contract.
commit 6bad87749a321b8fd7cec83384f7582072fabbd6
Author: Dawid Weiss <dw...@...>
Date: 2018-08-03T10:00:42Z
Wrong imports corrected.
commit 689d35c873dec8e805b4d37e7d528ac7f2a820df
Author: Dawid Weiss <dw...@...>
Date: 2018-08-03T10:15:18Z
Plug ByteBuffersDirectory into LuceneTestCase.
commit d38c30092c0f474d019986a05fa6e9ad9343d869
Author: Dawid Weiss <dw...@...>
Date: 2018-08-03T10:26:08Z
Remove RamFile from PrimaryNode.
commit c9de116392b87a9cad62178bfd88debaa846abf7
Author: Dawid Weiss <dw...@...>
Date: 2018-08-03T10:53:23Z
Merge remote-tracking branch 'origin/master' into LUCENE-8438
commit 7301721ead72306a32c9241d5f622ea750b1c09d
Author: Dawid Weiss <dw...@...>
Date: 2018-08-03T11:43:11Z
Renamed tests. Extracted parent class for DataOutput subclasses.
commit 44691a5e9ea5474eeaa5faa358d4a37331831299
Author: Dawid Weiss <dw...@...>
Date: 2018-08-03T12:36:13Z
Buffer reuse for BBDataOutput.
commit c8bd720b102dc850fdae847ad20c9cfaabdbcda0
Author: Dawid Weiss <dw...@...>
Date: 2018-08-03T12:46:05Z
Removed RAMFile dependency in SortingLeafReader.
commit 11d6149fe4f6b61e5aad9824f7cf36806e942820
Author: Dawid Weiss <da...@...>
Date: 2018-08-03T19:59:18Z
Merge remote-tracking branch 'origin/master' into LUCENE-8438
commit 27d1c4d55863d4b26658dbaa4eaddfbda44a4674
Author: Dawid Weiss <da...@...>
Date: 2018-08-03T20:27:03Z
Removed RAMFile from PrefixCodedTerms.
commit e8d2e8c969399f2c59ec260d3f6e174956485335
Author: Dawid Weiss <da...@...>
Date: 2018-08-03T20:42:47Z
RAMFile removal from test.
commit a1d556371afd72384160379cbebc75eb5929cb79
Author: Dawid Weiss <da...@...>
Date: 2018-08-03T20:46:21Z
Ignore non-passing test.
commit 37d6372d1ebf3473ca398e6ddf99f9d73a5bed54
Author: Dawid Weiss <da...@...>
Date: 2018-08-04T20:05:46Z
Removal of RAMFile from Lucene70DocValuesFormat.
commit 38959f25a208d2b434c77bbfe1ef6935ea9db812
Author: Dawid Weiss <da...@...>
Date: 2018-08-05T20:55:06Z
More replacements of ram input streams.
commit e98dba7848e9045967c5c48158b4191207ae4053
Author: Dawid Weiss <dw...@...>
Date: 2018-08-06T08:18:10Z
Make checksum calculation lazy.
commit a5a346e2676720d8fb86edb2c7e1ef9250aa2885
Author: Dawid Weiss <dw...@...>
Date: 2018-08-06T08:22:50Z
Replacement of RamOutputStream.
commit 99f0d3c0f0e3c234b2ce0bbbee53f6cc951ada32
Author: Dawid Weiss <dw...@...>
Date: 2018-08-06T08:51:22Z
Make reset() parameterless, sigh. Makes refactoring easier.
commit 7a8955609174ddb9edd62d71a91a72b509eef14b
Author: Dawid Weiss <dw...@...>
Date: 2018-08-06T08:56:19Z
Replacement of RamOutputStream.
commit c65fea31e4726968681e78e5f0f67fb1c00eb45b
Author: Dawid Weiss <dw...@...>
Date: 2018-08-06T09:12:45Z
Replacement of RamOutputStream.
commit b5d8e3f84ac8c3c0dd50a2d16030708bea4a467c
Author: Dawid Weiss <dw...@...>
Date: 2018-08-06T09:19:42Z
Replacement of RamOutputStream.
commit e5b7ae4f89fa4827ba10615061d5108df064a35c
Author: Dawid Weiss <dw...@...>
Date: 2018-08-06T09:25:47Z
Replacement of RamOutputStream.
commit a086e7811a22431f19bbec9054a622122a9207b5
Author: Dawid Weiss <dw...@...>
Date: 2018-08-06T09:28:18Z
Replacement of RamOutputStream.
commit 9c2beb536f15ea0e73f63e832c46f2d4ce3bd70e
Author: Dawid Weiss <dw...@...>
Date: 2018-08-06T09:39:26Z
Replacement of RamOutputStream.
commit 265c2359c72f6f39aee42b06281d596b04c77a55
Author: Dawid Weiss <dw...@...>
Date: 2018-08-06T09:48:28Z
Replacement of RamOutputStream.
commit 8dfb8a7f681548d838a3e412a1b9e4d369236e77
Author: Dawid Weiss <dw...@...>
Date: 2018-08-06T09:50:41Z
Add ByteBuffersDirectory to spins().
----
---
---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@lucene.apache.org
For additional commands, e-mail: dev-help@lucene.apache.org
[GitHub] lucene-solr pull request #432: LUCENE-8438: RAMDirectory speed improvements ...
Posted by uschindler <gi...@git.apache.org>.
Github user uschindler commented on a diff in the pull request:
https://github.com/apache/lucene-solr/pull/432#discussion_r208520676
--- Diff: lucene/core/src/java/org/apache/lucene/store/ByteBuffersDirectory.java ---
@@ -0,0 +1,237 @@
+package org.apache.lucene.store;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.file.AccessDeniedException;
+import java.nio.file.FileAlreadyExistsException;
+import java.nio.file.NoSuchFileException;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+import java.util.Locale;
+import java.util.Objects;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.function.BiFunction;
+import java.util.function.Function;
+import java.util.zip.CRC32;
+
+import org.apache.lucene.index.IndexFileNames;
+import org.apache.lucene.util.BitUtil;
+
+public final class ByteBuffersDirectory extends BaseDirectory {
+ public static final BiFunction<String, ByteBuffersDataOutput, IndexInput> OUTPUT_AS_MANY_BUFFERS =
+ (fileName, output) -> {
+ ByteBuffersDataInput dataInput = output.toDataInput();
+ String inputName = String.format(Locale.ROOT, "%s (file=%s, buffers=%s)",
+ ByteBuffersIndexInput.class.getSimpleName(),
+ fileName,
+ dataInput.toString());
+ return new ByteBuffersIndexInput(dataInput, inputName);
+ };
+
+ public static final BiFunction<String, ByteBuffersDataOutput, IndexInput> OUTPUT_AS_ONE_BUFFER =
+ (fileName, output) -> {
+ ByteBuffersDataInput dataInput = new ByteBuffersDataInput(Arrays.asList(ByteBuffer.wrap(output.copyToArray())));
+ String inputName = String.format(Locale.ROOT, "%s (file=%s, buffers=%s)",
+ ByteBuffersIndexInput.class.getSimpleName(),
+ fileName,
+ dataInput.toString());
+ return new ByteBuffersIndexInput(dataInput, inputName);
+ };
+
+ public static final BiFunction<String, ByteBuffersDataOutput, IndexInput> OUTPUT_AS_BYTE_ARRAY =
+ (fileName, output) -> {
+ byte[] array = output.copyToArray();
+ String inputName = String.format(Locale.ROOT, "%s (file=%s, length=%s)",
+ ByteArrayIndexInput.class.getSimpleName(),
+ fileName,
+ array.length);
+ return new ByteArrayIndexInput(inputName, array, 0, array.length);
+ };
+
+ public static final BiFunction<String, ByteBuffersDataOutput, IndexInput> OUTPUT_AS_MANY_BUFFERS_LUCENE =
+ (fileName, output) -> {
+ List<ByteBuffer> bufferList = output.toBufferList();
+ int chunkSizePower;
+ bufferList.add(ByteBuffer.allocate(0));
--- End diff --
why do we need the empty buffer
---
---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@lucene.apache.org
For additional commands, e-mail: dev-help@lucene.apache.org
[GitHub] lucene-solr pull request #432: LUCENE-8438: RAMDirectory speed improvements ...
Posted by dweiss <gi...@git.apache.org>.
Github user dweiss commented on a diff in the pull request:
https://github.com/apache/lucene-solr/pull/432#discussion_r208520234
--- Diff: lucene/core/src/java/org/apache/lucene/store/ByteBuffersDataOutput.java ---
@@ -0,0 +1,488 @@
+package org.apache.lucene.store;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import java.util.function.Consumer;
+import java.util.function.IntConsumer;
+import java.util.function.IntFunction;
+
+import org.apache.lucene.util.Accountable;
+import org.apache.lucene.util.BitUtil;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.RamUsageEstimator;
+import org.apache.lucene.util.UnicodeUtil;
+
+public final class ByteBuffersDataOutput extends DataOutput implements Accountable {
+ private final static ByteBuffer EMPTY = ByteBuffer.allocate(0);
+ private final static List<ByteBuffer> EMPTY_LIST = Arrays.asList(EMPTY);
+ private final static byte [] EMPTY_BYTE_ARRAY = {};
+
+ public final static IntFunction<ByteBuffer> ALLOCATE_BB_ON_HEAP = (size) -> {
--- End diff --
I'll try to pay attention to that, thanks for looking!
---
---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@lucene.apache.org
For additional commands, e-mail: dev-help@lucene.apache.org
[GitHub] lucene-solr pull request #432: LUCENE-8438: RAMDirectory speed improvements ...
Posted by dweiss <gi...@git.apache.org>.
Github user dweiss commented on a diff in the pull request:
https://github.com/apache/lucene-solr/pull/432#discussion_r208516834
--- Diff: lucene/core/src/java/org/apache/lucene/store/ByteBuffersDataOutput.java ---
@@ -0,0 +1,488 @@
+package org.apache.lucene.store;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import java.util.function.Consumer;
+import java.util.function.IntConsumer;
+import java.util.function.IntFunction;
+
+import org.apache.lucene.util.Accountable;
+import org.apache.lucene.util.BitUtil;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.RamUsageEstimator;
+import org.apache.lucene.util.UnicodeUtil;
+
+public final class ByteBuffersDataOutput extends DataOutput implements Accountable {
+ private final static ByteBuffer EMPTY = ByteBuffer.allocate(0);
+ private final static List<ByteBuffer> EMPTY_LIST = Arrays.asList(EMPTY);
+ private final static byte [] EMPTY_BYTE_ARRAY = {};
+
+ public final static IntFunction<ByteBuffer> ALLOCATE_BB_ON_HEAP = (size) -> {
--- End diff --
Ah, sure. It's just easier for me to see what the argument is sometimes. I'll correct.
---
---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@lucene.apache.org
For additional commands, e-mail: dev-help@lucene.apache.org
[GitHub] lucene-solr pull request #432: LUCENE-8438: RAMDirectory speed improvements ...
Posted by uschindler <gi...@git.apache.org>.
Github user uschindler commented on a diff in the pull request:
https://github.com/apache/lucene-solr/pull/432#discussion_r209170816
--- Diff: lucene/core/src/java/org/apache/lucene/store/ByteBuffersDirectory.java ---
@@ -0,0 +1,237 @@
+package org.apache.lucene.store;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.file.AccessDeniedException;
+import java.nio.file.FileAlreadyExistsException;
+import java.nio.file.NoSuchFileException;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+import java.util.Locale;
+import java.util.Objects;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.function.BiFunction;
+import java.util.function.Function;
+import java.util.zip.CRC32;
+
+import org.apache.lucene.index.IndexFileNames;
+import org.apache.lucene.util.BitUtil;
+
+public final class ByteBuffersDirectory extends BaseDirectory {
+ public static final BiFunction<String, ByteBuffersDataOutput, IndexInput> OUTPUT_AS_MANY_BUFFERS =
+ (fileName, output) -> {
+ ByteBuffersDataInput dataInput = output.toDataInput();
+ String inputName = String.format(Locale.ROOT, "%s (file=%s, buffers=%s)",
+ ByteBuffersIndexInput.class.getSimpleName(),
+ fileName,
+ dataInput.toString());
+ return new ByteBuffersIndexInput(dataInput, inputName);
+ };
+
+ public static final BiFunction<String, ByteBuffersDataOutput, IndexInput> OUTPUT_AS_ONE_BUFFER =
+ (fileName, output) -> {
+ ByteBuffersDataInput dataInput = new ByteBuffersDataInput(Arrays.asList(ByteBuffer.wrap(output.copyToArray())));
+ String inputName = String.format(Locale.ROOT, "%s (file=%s, buffers=%s)",
+ ByteBuffersIndexInput.class.getSimpleName(),
+ fileName,
+ dataInput.toString());
+ return new ByteBuffersIndexInput(dataInput, inputName);
+ };
+
+ public static final BiFunction<String, ByteBuffersDataOutput, IndexInput> OUTPUT_AS_BYTE_ARRAY =
+ (fileName, output) -> {
+ byte[] array = output.copyToArray();
+ String inputName = String.format(Locale.ROOT, "%s (file=%s, length=%s)",
+ ByteArrayIndexInput.class.getSimpleName(),
+ fileName,
+ array.length);
+ return new ByteArrayIndexInput(inputName, array, 0, array.length);
+ };
+
+ public static final BiFunction<String, ByteBuffersDataOutput, IndexInput> OUTPUT_AS_MANY_BUFFERS_LUCENE =
+ (fileName, output) -> {
+ List<ByteBuffer> bufferList = output.toBufferList();
+ int chunkSizePower;
+ bufferList.add(ByteBuffer.allocate(0));
--- End diff --
What exception did you get and where?
---
---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@lucene.apache.org
For additional commands, e-mail: dev-help@lucene.apache.org
[GitHub] lucene-solr pull request #432: LUCENE-8438: RAMDirectory speed improvements ...
Posted by uschindler <gi...@git.apache.org>.
Github user uschindler commented on a diff in the pull request:
https://github.com/apache/lucene-solr/pull/432#discussion_r208518497
--- Diff: lucene/core/src/java/org/apache/lucene/store/ByteBuffersDataOutput.java ---
@@ -0,0 +1,488 @@
+package org.apache.lucene.store;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import java.util.function.Consumer;
+import java.util.function.IntConsumer;
+import java.util.function.IntFunction;
+
+import org.apache.lucene.util.Accountable;
+import org.apache.lucene.util.BitUtil;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.RamUsageEstimator;
+import org.apache.lucene.util.UnicodeUtil;
+
+public final class ByteBuffersDataOutput extends DataOutput implements Accountable {
+ private final static ByteBuffer EMPTY = ByteBuffer.allocate(0);
+ private final static List<ByteBuffer> EMPTY_LIST = Arrays.asList(EMPTY);
+ private final static byte [] EMPTY_BYTE_ARRAY = {};
+
+ public final static IntFunction<ByteBuffer> ALLOCATE_BB_ON_HEAP = (size) -> {
--- End diff --
There are other places with unneeded lambdas, but that's just a preference. I am fine with both.
---
---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@lucene.apache.org
For additional commands, e-mail: dev-help@lucene.apache.org
[GitHub] lucene-solr pull request #432: LUCENE-8438: RAMDirectory speed improvements ...
Posted by dweiss <gi...@git.apache.org>.
Github user dweiss commented on a diff in the pull request:
https://github.com/apache/lucene-solr/pull/432#discussion_r209172193
--- Diff: lucene/core/src/java/org/apache/lucene/store/ByteBuffersDirectory.java ---
@@ -0,0 +1,237 @@
+package org.apache.lucene.store;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.file.AccessDeniedException;
+import java.nio.file.FileAlreadyExistsException;
+import java.nio.file.NoSuchFileException;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+import java.util.Locale;
+import java.util.Objects;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.function.BiFunction;
+import java.util.function.Function;
+import java.util.zip.CRC32;
+
+import org.apache.lucene.index.IndexFileNames;
+import org.apache.lucene.util.BitUtil;
+
+public final class ByteBuffersDirectory extends BaseDirectory {
+ public static final BiFunction<String, ByteBuffersDataOutput, IndexInput> OUTPUT_AS_MANY_BUFFERS =
+ (fileName, output) -> {
+ ByteBuffersDataInput dataInput = output.toDataInput();
+ String inputName = String.format(Locale.ROOT, "%s (file=%s, buffers=%s)",
+ ByteBuffersIndexInput.class.getSimpleName(),
+ fileName,
+ dataInput.toString());
+ return new ByteBuffersIndexInput(dataInput, inputName);
+ };
+
+ public static final BiFunction<String, ByteBuffersDataOutput, IndexInput> OUTPUT_AS_ONE_BUFFER =
+ (fileName, output) -> {
+ ByteBuffersDataInput dataInput = new ByteBuffersDataInput(Arrays.asList(ByteBuffer.wrap(output.copyToArray())));
+ String inputName = String.format(Locale.ROOT, "%s (file=%s, buffers=%s)",
+ ByteBuffersIndexInput.class.getSimpleName(),
+ fileName,
+ dataInput.toString());
+ return new ByteBuffersIndexInput(dataInput, inputName);
+ };
+
+ public static final BiFunction<String, ByteBuffersDataOutput, IndexInput> OUTPUT_AS_BYTE_ARRAY =
+ (fileName, output) -> {
+ byte[] array = output.copyToArray();
+ String inputName = String.format(Locale.ROOT, "%s (file=%s, length=%s)",
+ ByteArrayIndexInput.class.getSimpleName(),
+ fileName,
+ array.length);
+ return new ByteArrayIndexInput(inputName, array, 0, array.length);
+ };
+
+ public static final BiFunction<String, ByteBuffersDataOutput, IndexInput> OUTPUT_AS_MANY_BUFFERS_LUCENE =
+ (fileName, output) -> {
+ List<ByteBuffer> bufferList = output.toBufferList();
+ int chunkSizePower;
+ bufferList.add(ByteBuffer.allocate(0));
--- End diff --
I know -- it's the wraparound case in case pos falls just outside of the block boundary. Simplifies some logic later on, I think it's harmless.
---
---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@lucene.apache.org
For additional commands, e-mail: dev-help@lucene.apache.org
[GitHub] lucene-solr pull request #432: LUCENE-8438: RAMDirectory speed improvements ...
Posted by uschindler <gi...@git.apache.org>.
Github user uschindler commented on a diff in the pull request:
https://github.com/apache/lucene-solr/pull/432#discussion_r208515533
--- Diff: lucene/core/src/java/org/apache/lucene/store/ByteBuffersDataOutput.java ---
@@ -0,0 +1,488 @@
+package org.apache.lucene.store;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import java.util.function.Consumer;
+import java.util.function.IntConsumer;
+import java.util.function.IntFunction;
+
+import org.apache.lucene.util.Accountable;
+import org.apache.lucene.util.BitUtil;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.RamUsageEstimator;
+import org.apache.lucene.util.UnicodeUtil;
+
+public final class ByteBuffersDataOutput extends DataOutput implements Accountable {
+ private final static ByteBuffer EMPTY = ByteBuffer.allocate(0);
+ private final static List<ByteBuffer> EMPTY_LIST = Arrays.asList(EMPTY);
+ private final static byte [] EMPTY_BYTE_ARRAY = {};
+
+ public final static IntFunction<ByteBuffer> ALLOCATE_BB_ON_HEAP = (size) -> {
--- End diff --
Just write `public final static IntFunction<ByteBuffer> ALLOCATE_BB_ON_HEAP = ByteBuffer::allocate;`, no lambda needed. The same applies at other places in the PR, but this looks really strange.
---
---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@lucene.apache.org
For additional commands, e-mail: dev-help@lucene.apache.org
[GitHub] lucene-solr pull request #432: LUCENE-8438: RAMDirectory speed improvements ...
Posted by dweiss <gi...@git.apache.org>.
Github user dweiss commented on a diff in the pull request:
https://github.com/apache/lucene-solr/pull/432#discussion_r208522104
--- Diff: lucene/core/src/java/org/apache/lucene/store/ByteBuffersDirectory.java ---
@@ -0,0 +1,237 @@
+package org.apache.lucene.store;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.file.AccessDeniedException;
+import java.nio.file.FileAlreadyExistsException;
+import java.nio.file.NoSuchFileException;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+import java.util.Locale;
+import java.util.Objects;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.function.BiFunction;
+import java.util.function.Function;
+import java.util.zip.CRC32;
+
+import org.apache.lucene.index.IndexFileNames;
+import org.apache.lucene.util.BitUtil;
+
+public final class ByteBuffersDirectory extends BaseDirectory {
+ public static final BiFunction<String, ByteBuffersDataOutput, IndexInput> OUTPUT_AS_MANY_BUFFERS =
+ (fileName, output) -> {
+ ByteBuffersDataInput dataInput = output.toDataInput();
+ String inputName = String.format(Locale.ROOT, "%s (file=%s, buffers=%s)",
+ ByteBuffersIndexInput.class.getSimpleName(),
+ fileName,
+ dataInput.toString());
+ return new ByteBuffersIndexInput(dataInput, inputName);
+ };
+
+ public static final BiFunction<String, ByteBuffersDataOutput, IndexInput> OUTPUT_AS_ONE_BUFFER =
+ (fileName, output) -> {
+ ByteBuffersDataInput dataInput = new ByteBuffersDataInput(Arrays.asList(ByteBuffer.wrap(output.copyToArray())));
+ String inputName = String.format(Locale.ROOT, "%s (file=%s, buffers=%s)",
+ ByteBuffersIndexInput.class.getSimpleName(),
+ fileName,
+ dataInput.toString());
+ return new ByteBuffersIndexInput(dataInput, inputName);
+ };
+
+ public static final BiFunction<String, ByteBuffersDataOutput, IndexInput> OUTPUT_AS_BYTE_ARRAY =
+ (fileName, output) -> {
+ byte[] array = output.copyToArray();
+ String inputName = String.format(Locale.ROOT, "%s (file=%s, length=%s)",
+ ByteArrayIndexInput.class.getSimpleName(),
+ fileName,
+ array.length);
+ return new ByteArrayIndexInput(inputName, array, 0, array.length);
+ };
+
+ public static final BiFunction<String, ByteBuffersDataOutput, IndexInput> OUTPUT_AS_MANY_BUFFERS_LUCENE =
+ (fileName, output) -> {
+ List<ByteBuffer> bufferList = output.toBufferList();
+ int chunkSizePower;
+ bufferList.add(ByteBuffer.allocate(0));
--- End diff --
Lucene's implementation makes an assumption about this, actually (if multiple buffers are used). Try to remove it and run the tests, you'll see.
---
---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@lucene.apache.org
For additional commands, e-mail: dev-help@lucene.apache.org
[GitHub] lucene-solr pull request #432: LUCENE-8438: RAMDirectory speed improvements ...
Posted by uschindler <gi...@git.apache.org>.
Github user uschindler commented on a diff in the pull request:
https://github.com/apache/lucene-solr/pull/432#discussion_r209170130
--- Diff: lucene/core/src/java/org/apache/lucene/store/ByteBuffersDirectory.java ---
@@ -0,0 +1,237 @@
+package org.apache.lucene.store;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.file.AccessDeniedException;
+import java.nio.file.FileAlreadyExistsException;
+import java.nio.file.NoSuchFileException;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+import java.util.Locale;
+import java.util.Objects;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.function.BiFunction;
+import java.util.function.Function;
+import java.util.zip.CRC32;
+
+import org.apache.lucene.index.IndexFileNames;
+import org.apache.lucene.util.BitUtil;
+
+public final class ByteBuffersDirectory extends BaseDirectory {
+ public static final BiFunction<String, ByteBuffersDataOutput, IndexInput> OUTPUT_AS_MANY_BUFFERS =
+ (fileName, output) -> {
+ ByteBuffersDataInput dataInput = output.toDataInput();
+ String inputName = String.format(Locale.ROOT, "%s (file=%s, buffers=%s)",
+ ByteBuffersIndexInput.class.getSimpleName(),
+ fileName,
+ dataInput.toString());
+ return new ByteBuffersIndexInput(dataInput, inputName);
+ };
+
+ public static final BiFunction<String, ByteBuffersDataOutput, IndexInput> OUTPUT_AS_ONE_BUFFER =
+ (fileName, output) -> {
+ ByteBuffersDataInput dataInput = new ByteBuffersDataInput(Arrays.asList(ByteBuffer.wrap(output.copyToArray())));
+ String inputName = String.format(Locale.ROOT, "%s (file=%s, buffers=%s)",
+ ByteBuffersIndexInput.class.getSimpleName(),
+ fileName,
+ dataInput.toString());
+ return new ByteBuffersIndexInput(dataInput, inputName);
+ };
+
+ public static final BiFunction<String, ByteBuffersDataOutput, IndexInput> OUTPUT_AS_BYTE_ARRAY =
+ (fileName, output) -> {
+ byte[] array = output.copyToArray();
+ String inputName = String.format(Locale.ROOT, "%s (file=%s, length=%s)",
+ ByteArrayIndexInput.class.getSimpleName(),
+ fileName,
+ array.length);
+ return new ByteArrayIndexInput(inputName, array, 0, array.length);
+ };
+
+ public static final BiFunction<String, ByteBuffersDataOutput, IndexInput> OUTPUT_AS_MANY_BUFFERS_LUCENE =
+ (fileName, output) -> {
+ List<ByteBuffer> bufferList = output.toBufferList();
+ int chunkSizePower;
+ bufferList.add(ByteBuffer.allocate(0));
--- End diff --
I think the 0-byte buffer is only needed for buffers with exactly the chunkSize, see the comment:
```java
// we always allocate one more buffer, the last one may be a 0 byte one
final int nrBuffers = (int) (length >>> chunkSizePower) + 1;
```
So the last one is a 0-bytes buffer, if the total size is actually exactly 2^x. Maybe we can fix this, but this made calculations more easy.
---
---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@lucene.apache.org
For additional commands, e-mail: dev-help@lucene.apache.org