-
Notifications
You must be signed in to change notification settings - Fork 20
/
Copy pathZstdCompressionMode.java
209 lines (167 loc) · 7.7 KB
/
ZstdCompressionMode.java
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
/*
* SPDX-License-Identifier: Apache-2.0
*
* The OpenSearch Contributors require contributions made to
* this file be licensed under the Apache-2.0 license or a
* compatible open source license.
*/
package org.opensearch.index.codec.customcodecs;
import com.github.luben.zstd.Zstd;
import com.github.luben.zstd.ZstdCompressCtx;
import com.github.luben.zstd.ZstdDecompressCtx;
import com.github.luben.zstd.ZstdDictCompress;
import com.github.luben.zstd.ZstdDictDecompress;
import org.apache.lucene.codecs.compressing.CompressionMode;
import org.apache.lucene.codecs.compressing.Compressor;
import org.apache.lucene.codecs.compressing.Decompressor;
import org.apache.lucene.store.ByteBuffersDataInput;
import org.apache.lucene.store.DataInput;
import org.apache.lucene.store.DataOutput;
import org.apache.lucene.util.ArrayUtil;
import org.apache.lucene.util.BytesRef;
import java.io.IOException;
/** Zstandard Compression Mode */
public class ZstdCompressionMode extends CompressionMode {
private static final int NUM_SUB_BLOCKS = 10;
private static final int DICT_SIZE_FACTOR = 6;
private static final int DEFAULT_COMPRESSION_LEVEL = 6;
private final int compressionLevel;
/** default constructor */
protected ZstdCompressionMode() {
this.compressionLevel = DEFAULT_COMPRESSION_LEVEL;
}
/**
* Creates a new instance.
*
* @param compressionLevel The compression level to use.
*/
protected ZstdCompressionMode(int compressionLevel) {
this.compressionLevel = compressionLevel;
}
/** Creates a new compressor instance.*/
@Override
public Compressor newCompressor() {
return new ZstdCompressor(compressionLevel);
}
/** Creates a new decompressor instance. */
@Override
public Decompressor newDecompressor() {
return new ZstdDecompressor();
}
/** zstandard compressor */
private static final class ZstdCompressor extends Compressor {
private final int compressionLevel;
private byte[] compressedBuffer;
/** compressor with a given compresion level */
public ZstdCompressor(int compressionLevel) {
this.compressionLevel = compressionLevel;
compressedBuffer = BytesRef.EMPTY_BYTES;
}
/*resuable compress function*/
private void doCompress(byte[] bytes, int offset, int length, ZstdCompressCtx cctx, DataOutput out) throws IOException {
if (length == 0) {
out.writeVInt(0);
return;
}
final int maxCompressedLength = (int) Zstd.compressBound(length);
compressedBuffer = ArrayUtil.growNoCopy(compressedBuffer, maxCompressedLength);
int compressedSize = cctx.compressByteArray(compressedBuffer, 0, compressedBuffer.length, bytes, offset, length);
out.writeVInt(compressedSize);
out.writeBytes(compressedBuffer, compressedSize);
}
private void compress(byte[] bytes, int offset, int length, DataOutput out) throws IOException {
assert offset >= 0 : "offset value must be greater than 0";
final int dictLength = length / (NUM_SUB_BLOCKS * DICT_SIZE_FACTOR);
final int blockLength = (length - dictLength + NUM_SUB_BLOCKS - 1) / NUM_SUB_BLOCKS;
out.writeVInt(dictLength);
out.writeVInt(blockLength);
final int end = offset + length;
assert end >= 0 : "buffer read size must be greater than 0";
try (ZstdCompressCtx cctx = new ZstdCompressCtx()) {
cctx.setLevel(compressionLevel);
// dictionary compression first
doCompress(bytes, offset, dictLength, cctx, out);
try (ZstdDictCompress dictCompress = new ZstdDictCompress(bytes, offset, dictLength, compressionLevel)) {
cctx.loadDict(dictCompress);
for (int start = offset + dictLength; start < end; start += blockLength) {
int l = Math.min(blockLength, end - start);
doCompress(bytes, start, l, cctx, out);
}
}
}
}
@Override
public void compress(ByteBuffersDataInput buffersInput, DataOutput out) throws IOException {
final int length = (int) buffersInput.size();
byte[] bytes = new byte[length];
buffersInput.readBytes(bytes, 0, length);
compress(bytes, 0, length, out);
}
@Override
public void close() throws IOException {}
}
/** zstandard decompressor */
private static final class ZstdDecompressor extends Decompressor {
private byte[] compressedBuffer;
/** default decompressor */
public ZstdDecompressor() {
compressedBuffer = BytesRef.EMPTY_BYTES;
}
/*resuable decompress function*/
private void doDecompress(DataInput in, ZstdDecompressCtx dctx, BytesRef bytes, int decompressedLen) throws IOException {
final int compressedLength = in.readVInt();
if (compressedLength == 0) {
return;
}
compressedBuffer = ArrayUtil.growNoCopy(compressedBuffer, compressedLength);
in.readBytes(compressedBuffer, 0, compressedLength);
bytes.bytes = ArrayUtil.grow(bytes.bytes, bytes.length + decompressedLen);
int uncompressed = dctx.decompressByteArray(bytes.bytes, bytes.length, decompressedLen, compressedBuffer, 0, compressedLength);
if (decompressedLen != uncompressed) {
throw new IllegalStateException(decompressedLen + " " + uncompressed);
}
bytes.length += uncompressed;
}
@Override
public void decompress(DataInput in, int originalLength, int offset, int length, BytesRef bytes) throws IOException {
assert offset + length <= originalLength : "buffer read size must be within limit";
if (length == 0) {
bytes.length = 0;
return;
}
final int dictLength = in.readVInt();
final int blockLength = in.readVInt();
bytes.bytes = ArrayUtil.growNoCopy(bytes.bytes, dictLength);
bytes.offset = bytes.length = 0;
try (ZstdDecompressCtx dctx = new ZstdDecompressCtx()) {
// decompress dictionary first
doDecompress(in, dctx, bytes, dictLength);
try (ZstdDictDecompress dictDecompress = new ZstdDictDecompress(bytes.bytes, 0, dictLength)) {
dctx.loadDict(dictDecompress);
int offsetInBlock = dictLength;
int offsetInBytesRef = offset;
// Skip unneeded blocks
while (offsetInBlock + blockLength < offset) {
final int compressedLength = in.readVInt();
in.skipBytes(compressedLength);
offsetInBlock += blockLength;
offsetInBytesRef -= blockLength;
}
// Read blocks that intersect with the interval we need
while (offsetInBlock < offset + length) {
int l = Math.min(blockLength, originalLength - offsetInBlock);
doDecompress(in, dctx, bytes, l);
offsetInBlock += blockLength;
}
bytes.offset = offsetInBytesRef;
bytes.length = length;
assert bytes.isValid() : "decompression output is corrupted";
}
}
}
@Override
public Decompressor clone() {
return new ZstdDecompressor();
}
}
}