From 28f43e3b565398ea8ac0ffe7c17ac031fc203ad0 Mon Sep 17 00:00:00 2001
From: Robert Muir
Date: Mon, 11 Jun 2012 16:12:12 +0000
Subject: [PATCH] LUCENE-4129: add codecheader to .frq/.prx
git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@1348927 13f79535-47bb-0310-9956-ffa450edef68
---
.../lucene40/Lucene40PostingsFormat.java | 6 +-
.../lucene40/Lucene40PostingsReader.java | 49 +++++----
.../lucene40/Lucene40PostingsWriter.java | 12 ++-
.../codecs/pulsing/PulsingPostingsFormat.java | 21 ++--
.../lucene40/TestAllFilesHaveCodecHeader.java | 98 ++++++++++++++++++
.../org/apache/lucene/index/index.40.cfs.zip | Bin 9081 -> 9175 bytes
.../apache/lucene/index/index.40.nocfs.zip | Bin 18257 -> 17101 bytes
.../lucene/index/index.40.optimized.cfs.zip | Bin 2829 -> 2975 bytes
.../lucene/index/index.40.optimized.nocfs.zip | Bin 4489 -> 4854 bytes
.../NestedPulsingPostingsFormat.java | 25 +++--
10 files changed, 169 insertions(+), 42 deletions(-)
create mode 100644 lucene/core/src/test/org/apache/lucene/codecs/lucene40/TestAllFilesHaveCodecHeader.java
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40PostingsFormat.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40PostingsFormat.java
index 76f11b4114d..eb90cbc8c73 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40PostingsFormat.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40PostingsFormat.java
@@ -159,7 +159,8 @@ import org.apache.lucene.util.fst.FST; // javadocs
* with the frequency of the term in that document (except when frequencies are
* omitted: {@link IndexOptions#DOCS_ONLY}).
*
- * - FreqFile (.frq) --> <TermFreqs, SkipData> TermCount
+ * - FreqFile (.frq) --> Header, <TermFreqs, SkipData> TermCount
+ * - Header --> {@link CodecUtil#writeHeader CodecHeader}
* - TermFreqs --> <TermFreq> DocFreq
* - TermFreq --> DocDelta[, Freq?]
* - SkipData --> <<SkipLevelLength, SkipLevel>
@@ -232,7 +233,8 @@ import org.apache.lucene.util.fst.FST; // javadocs
* anything into this file, and if all fields in the index omit positional data
* then the .prx file will not exist.
*
- * - ProxFile (.prx) --> <TermPositions> TermCount
+ * - ProxFile (.prx) --> Header, <TermPositions> TermCount
+ * - Header --> {@link CodecUtil#writeHeader CodecHeader}
* - TermPositions --> <Positions> DocFreq
* - Positions --> <PositionDelta,PayloadLength?,OffsetDelta?,OffsetLength?,PayloadData?> Freq
* - PositionDelta,OffsetDelta,OffsetLength,PayloadLength --> {@link DataOutput#writeVInt VInt}
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40PostingsReader.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40PostingsReader.java
index 576727482e6..a212dbcb884 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40PostingsReader.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40PostingsReader.java
@@ -37,6 +37,7 @@ import org.apache.lucene.util.ArrayUtil;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.CodecUtil;
+import org.apache.lucene.util.IOUtils;
/**
* Concrete class that reads the 4.0 frq/prox
@@ -58,29 +59,35 @@ public class Lucene40PostingsReader extends PostingsReaderBase {
// private String segment;
public Lucene40PostingsReader(Directory dir, FieldInfos fieldInfos, SegmentInfo segmentInfo, IOContext ioContext, String segmentSuffix) throws IOException {
- freqIn = dir.openInput(IndexFileNames.segmentFileName(segmentInfo.name, segmentSuffix, Lucene40PostingsFormat.FREQ_EXTENSION),
+ boolean success = false;
+ IndexInput freqIn = null;
+ IndexInput proxIn = null;
+ try {
+ freqIn = dir.openInput(IndexFileNames.segmentFileName(segmentInfo.name, segmentSuffix, Lucene40PostingsFormat.FREQ_EXTENSION),
ioContext);
- // TODO: hasProx should (somehow!) become codec private,
- // but it's tricky because 1) FIS.hasProx is global (it
- // could be all fields that have prox are written by a
- // different codec), 2) the field may have had prox in
- // the past but all docs w/ that field were deleted.
- // Really we'd need to init prxOut lazily on write, and
- // then somewhere record that we actually wrote it so we
- // know whether to open on read:
- if (fieldInfos.hasProx()) {
- boolean success = false;
- try {
+ CodecUtil.checkHeader(freqIn, Lucene40PostingsWriter.FRQ_CODEC, Lucene40PostingsWriter.VERSION_START,Lucene40PostingsWriter.VERSION_START);
+ // TODO: hasProx should (somehow!) become codec private,
+ // but it's tricky because 1) FIS.hasProx is global (it
+ // could be all fields that have prox are written by a
+ // different codec), 2) the field may have had prox in
+ // the past but all docs w/ that field were deleted.
+ // Really we'd need to init prxOut lazily on write, and
+ // then somewhere record that we actually wrote it so we
+ // know whether to open on read:
+ if (fieldInfos.hasProx()) {
proxIn = dir.openInput(IndexFileNames.segmentFileName(segmentInfo.name, segmentSuffix, Lucene40PostingsFormat.PROX_EXTENSION),
- ioContext);
- success = true;
- } finally {
- if (!success) {
- freqIn.close();
- }
+ ioContext);
+ CodecUtil.checkHeader(proxIn, Lucene40PostingsWriter.PRX_CODEC, Lucene40PostingsWriter.VERSION_START,Lucene40PostingsWriter.VERSION_START);
+ } else {
+ proxIn = null;
+ }
+ this.freqIn = freqIn;
+ this.proxIn = proxIn;
+ success = true;
+ } finally {
+ if (!success) {
+ IOUtils.closeWhileHandlingException(freqIn, proxIn);
}
- } else {
- proxIn = null;
}
}
@@ -88,7 +95,7 @@ public class Lucene40PostingsReader extends PostingsReaderBase {
public void init(IndexInput termsIn) throws IOException {
// Make sure we are talking to the matching past writer
- CodecUtil.checkHeader(termsIn, Lucene40PostingsWriter.CODEC,
+ CodecUtil.checkHeader(termsIn, Lucene40PostingsWriter.TERMS_CODEC,
Lucene40PostingsWriter.VERSION_START, Lucene40PostingsWriter.VERSION_START);
skipInterval = termsIn.readInt();
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40PostingsWriter.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40PostingsWriter.java
index f103897e573..36655662752 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40PostingsWriter.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40PostingsWriter.java
@@ -45,7 +45,9 @@ import org.apache.lucene.util.IOUtils;
* @lucene.experimental
*/
public final class Lucene40PostingsWriter extends PostingsWriterBase {
- final static String CODEC = "Lucene40PostingsWriter";
+ final static String TERMS_CODEC = "Lucene40PostingsWriterTerms";
+ final static String FRQ_CODEC = "Lucene40PostingsWriterFrq";
+ final static String PRX_CODEC = "Lucene40PostingsWriterPrx";
//private static boolean DEBUG = BlockTreeTermsWriter.DEBUG;
@@ -102,7 +104,9 @@ public final class Lucene40PostingsWriter extends PostingsWriterBase {
String fileName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, Lucene40PostingsFormat.FREQ_EXTENSION);
freqOut = state.directory.createOutput(fileName, state.context);
boolean success = false;
+ IndexOutput proxOut = null;
try {
+ CodecUtil.writeHeader(freqOut, FRQ_CODEC, VERSION_CURRENT);
// TODO: this is a best effort, if one of these fields has no postings
// then we make an empty prx file, same as if we are wrapped in
// per-field postingsformat. maybe... we shouldn't
@@ -112,14 +116,16 @@ public final class Lucene40PostingsWriter extends PostingsWriterBase {
// prox file
fileName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, Lucene40PostingsFormat.PROX_EXTENSION);
proxOut = state.directory.createOutput(fileName, state.context);
+ CodecUtil.writeHeader(proxOut, PRX_CODEC, VERSION_CURRENT);
} else {
// Every field omits TF so we will write no prox file
proxOut = null;
}
+ this.proxOut = proxOut;
success = true;
} finally {
if (!success) {
- IOUtils.closeWhileHandlingException(freqOut);
+ IOUtils.closeWhileHandlingException(freqOut, proxOut);
}
}
@@ -135,7 +141,7 @@ public final class Lucene40PostingsWriter extends PostingsWriterBase {
@Override
public void start(IndexOutput termsOut) throws IOException {
this.termsOut = termsOut;
- CodecUtil.writeHeader(termsOut, CODEC, VERSION_CURRENT);
+ CodecUtil.writeHeader(termsOut, TERMS_CODEC, VERSION_CURRENT);
termsOut.writeInt(skipInterval); // write skipInterval
termsOut.writeInt(maxSkipLevels); // write maxSkipLevels
termsOut.writeInt(skipMinimum); // write skipMinimum
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/pulsing/PulsingPostingsFormat.java b/lucene/core/src/java/org/apache/lucene/codecs/pulsing/PulsingPostingsFormat.java
index b443b619c7c..69bb152ab2f 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/pulsing/PulsingPostingsFormat.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/pulsing/PulsingPostingsFormat.java
@@ -29,6 +29,7 @@ import org.apache.lucene.codecs.PostingsReaderBase;
import org.apache.lucene.codecs.PostingsWriterBase;
import org.apache.lucene.index.SegmentReadState;
import org.apache.lucene.index.SegmentWriteState;
+import org.apache.lucene.util.IOUtils;
/** This postings format "inlines" the postings for terms that have
* low docFreq. It wraps another postings format, which is used for
@@ -65,33 +66,39 @@ public abstract class PulsingPostingsFormat extends PostingsFormat {
@Override
public FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
- PostingsWriterBase docsWriter = wrappedPostingsBaseFormat.postingsWriterBase(state);
+ PostingsWriterBase docsWriter = null;
// Terms that have <= freqCutoff number of docs are
// "pulsed" (inlined):
- PostingsWriterBase pulsingWriter = new PulsingPostingsWriter(freqCutoff, docsWriter);
+ PostingsWriterBase pulsingWriter = null;
// Terms dict
boolean success = false;
try {
+ docsWriter = wrappedPostingsBaseFormat.postingsWriterBase(state);
+
+ // Terms that have <= freqCutoff number of docs are
+ // "pulsed" (inlined):
+ pulsingWriter = new PulsingPostingsWriter(freqCutoff, docsWriter);
FieldsConsumer ret = new BlockTreeTermsWriter(state, pulsingWriter, minBlockSize, maxBlockSize);
success = true;
return ret;
} finally {
if (!success) {
- pulsingWriter.close();
+ IOUtils.closeWhileHandlingException(docsWriter, pulsingWriter);
}
}
}
@Override
public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException {
-
- PostingsReaderBase docsReader = wrappedPostingsBaseFormat.postingsReaderBase(state);
- PostingsReaderBase pulsingReader = new PulsingPostingsReader(docsReader);
+ PostingsReaderBase docsReader = null;
+ PostingsReaderBase pulsingReader = null;
boolean success = false;
try {
+ docsReader = wrappedPostingsBaseFormat.postingsReaderBase(state);
+ pulsingReader = new PulsingPostingsReader(docsReader);
FieldsProducer ret = new BlockTreeTermsReader(
state.dir, state.fieldInfos, state.segmentInfo.name,
pulsingReader,
@@ -102,7 +109,7 @@ public abstract class PulsingPostingsFormat extends PostingsFormat {
return ret;
} finally {
if (!success) {
- pulsingReader.close();
+ IOUtils.closeWhileHandlingException(docsReader, pulsingReader);
}
}
}
diff --git a/lucene/core/src/test/org/apache/lucene/codecs/lucene40/TestAllFilesHaveCodecHeader.java b/lucene/core/src/test/org/apache/lucene/codecs/lucene40/TestAllFilesHaveCodecHeader.java
new file mode 100644
index 00000000000..c8d90d9ac96
--- /dev/null
+++ b/lucene/core/src/test/org/apache/lucene/codecs/lucene40/TestAllFilesHaveCodecHeader.java
@@ -0,0 +1,98 @@
+package org.apache.lucene.codecs.lucene40;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.IOException;
+
+import org.apache.lucene.analysis.MockAnalyzer;
+import org.apache.lucene.codecs.Codec;
+import org.apache.lucene.document.Document;
+import org.apache.lucene.document.Field;
+import org.apache.lucene.index.IndexFileNames;
+import org.apache.lucene.index.IndexWriterConfig;
+import org.apache.lucene.index.RandomIndexWriter;
+import org.apache.lucene.store.CompoundFileDirectory;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.util.CodecUtil;
+import org.apache.lucene.util.IOUtils;
+import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util._TestUtil;
+
+/**
+ * Test that a plain Lucene40Codec puts codec headers in all files.
+ */
+public class TestAllFilesHaveCodecHeader extends LuceneTestCase {
+ public void test() throws Exception {
+ Directory dir = newDirectory();
+ IndexWriterConfig conf = newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random()));
+ conf.setCodec(Codec.forName("Lucene40"));
+ // riw should sometimes create docvalues fields, etc
+ RandomIndexWriter riw = new RandomIndexWriter(random(), dir, conf);
+ Document doc = new Document();
+ // these fields should sometimes get term vectors, etc
+ Field idField = newStringField("id", "", Field.Store.NO);
+ Field bodyField = newTextField("body", "", Field.Store.NO);
+ doc.add(idField);
+ doc.add(bodyField);
+ for (int i = 0; i < 100; i++) {
+ idField.setStringValue(Integer.toString(i));
+ bodyField.setStringValue(_TestUtil.randomUnicodeString(random()));
+ riw.addDocument(doc);
+ if (random().nextInt(7) == 0) {
+ riw.commit();
+ }
+ }
+ riw.close();
+ checkHeaders(dir);
+ dir.close();
+ }
+
+ private void checkHeaders(Directory dir) throws IOException {
+ for (String file : dir.listAll()) {
+ if (file.equals(IndexFileNames.SEGMENTS_GEN)) {
+ continue; // segments.gen has no header, thats ok
+ }
+ if (file.endsWith(IndexFileNames.COMPOUND_FILE_EXTENSION)) {
+ /* TODO: enable this after resolving LUCENE-4130
+ * CompoundFileDirectory cfsDir = new CompoundFileDirectory(dir, file, newIOContext(random()), false);
+ * checkHeaders(cfsDir); // recurse into cfs
+ * cfsDir.close();
+ */
+ continue; // .cfs has its own header... would be nice to fix
+ }
+ if (file.endsWith(IndexFileNames.COMPOUND_FILE_ENTRIES_EXTENSION)) {
+ continue; // .cfe has its own header... would be nice to fix
+ }
+ IndexInput in = null;
+ boolean success = false;
+ try {
+ in = dir.openInput(file, newIOContext(random()));
+ int val = in.readInt();
+ assertEquals(file + " has no codec header, instead found: " + val, CodecUtil.CODEC_MAGIC, val);
+ success = true;
+ } finally {
+ if (success) {
+ IOUtils.close(in);
+ } else {
+ IOUtils.closeWhileHandlingException(in);
+ }
+ }
+ }
+ }
+}
diff --git a/lucene/core/src/test/org/apache/lucene/index/index.40.cfs.zip b/lucene/core/src/test/org/apache/lucene/index/index.40.cfs.zip
index 57c836bb52fd7bb18c460b26cb979f1bada22086..2a2f858c2615d163d29ed0c06bca653d0b497ae7 100644
GIT binary patch
delta 6911
zcmaKxbzBr(*!LG$I+tbXSXvrEN(2_9yCno9r5i~dKtQ?{BqgOw1QZDoRF;xd5R?u9
zrCS7^g&Dojnxz2Bn6vvc^wbih|6d(`?4|Lt4H;I@LHx4$c
zg>Fen6iJHg9C8-Dhz82}`Al?(2nGm*1EL362=ZIo*&68MgRo-J!RVHh#0$9c1~~X@
z%i6i&z4N9}&@<6tmcQvP5@r|FQE)@o5Mvha;JU`OE=cZ;l!~tI^O
z1jx0T#ZfY1lX-*?YiqP&XaoOy9p@>(cTEsW0fRtqXn#xD>-UtkeEbMO7~mBw5a{A(
z%}B$UPJtpvO!zn}v)BMRXBf^Nj7Lr(<}smP-%+6XJ_S@DlDl9FQX#j=r_0fT5^&Ynvg1s+u=b!rAEuk9DQ
zg*1&*;Y-HT#Pn_74r4UR#Ji+9l;*|ZllKk=v}ypuDye5r0PnNy2Nnj8j?-jJ_USzj
zc5El*=Z2LO`w}B%oj-MQbd+e?#(SAph_<}i!o0e1NOC93yLMJ`7fURC+HyTAM{FqG
z(|-V3%w{cD*4!@_7qDMo?MJ+!&}{58xmo6?xkK1UAO|e_ceOK9-}mTx4W^G^6~tH4
zh;*1a4g}Q#tai!
zX--zLz18gY`g`EKOdWDRDx4Cf+8j$5k6PonK^KEVo6RI*ALdaV#!d?-+|sF0^K5kEqo
zp*$-AY+z4Cy=jGQkwfK%#;Q=3b$zqlxYPW)*fih!l_o*Hy@3xiCEmj`+r&U49!bX;5EX
z8sf`fJ{~NbNB*W6R|m8YxkVLCK&GsC-BQewR2z6>LJ%CP^YI4gD@C{q<}#V>PPB9f
z1&l%z3cpUVTbd7I6nuQOba%ROoV1F`M5g|p{mJ0>{JrbP7GeAFs{(raar{@eJBrdp
zBTX(#McZ=~Ke(1U`2%6TZJO_?&c;CjjpurS8g6?#ZD*J)K^}OeSYE(ibm+c@y_1%I
z-)lPHUU|BIp|}sfjuPN5A|!9=QBILGQU438ZBZ#@B-&2v07C=>aUK2nZN3}AJQ4}WEyq}UWs>9-vX7y;(>26Ra!Vp;wu3w2ikNEQ^9JewUYK-*UnbA3w!vf-tV~1CZ^s`t;l%e^|3WvQn=zK&fY#
zpuf=F5;Al7IU3z-ztDZ2^1tXlU!MLS=x#IHsGeu@p_PNe9!AQMXtzX778gJuJ_Su+
z!8n6ZuqRdEWMxX_N&uF`E*>YFD_+>8Xl~b=s(=LC0L~U~prsJjW-36>H3p}(yJH93
z{N!MAbc;>S)~x2F!*(Vs!Ww#vA;myNrtWobP{b%cbA!yV&kX}w1sW#oAH
zGH(MkcwO{)Q9X9idl9!Ksc37OpWS}J-fQL-8UK2Bn7Z7H?9$RF00`Pcs`|LQ%=ZI}
z2W_QBzWl(GMWe(064Pj$L#2%D_dy-pq8d;K&?57Ggs~O
zMs!{gvP(=-P$Z#2C3?`))OCsHw;HoR6dVlT~4PcU46ybxIG8%aVOtY-60jz%jw&
z7N)F`d-@NJ^rLyX*=bW@w5ejLTcf4mD_at)mp=QtI!@1Kv1n1q4{q2ys$!9uS*}LL
zx+cx3Xs|3dsc&2-USqqWp;_qCP|Dsp)X>gXY8mPptHTv|myy__KT=e#X+ZD6M2$ksxUiG3UWtJ1}t8Iye}TKRNk
zPdT0W?A-EQOv^cGqA{1dlZ*Rw%%Z53Tk%xm(dO=sJ~A{Zw=6)L+`T8S0fnm@l}d>O
zF+RSNyYZtz+L-K*nnXk$MFOFAiDuz7A^hAaeZ)1QG45hMpc(jTIX{&oj#pjYJCM~qtNx%3UyC#ljIws
zPLd|qeJ>Cox{CX0aO$B@F2Ll##YtGRKxJ64J|i$nEWlq!i>!`FU$6RX=$;UKbA_$(
zgDl))1Frndo2>K`1Ni>fRlH7DaA?VwX?K~&9~KVW>4y0{Z|L@BcS64}d%LG})Ti8X3r_m`2BOg72|!vjLo&xR*DL328osWZ1gI=Zk)
zhm1o(yM42-&7;0m8-U4l!%L^TW2aQrA}4gvn#Aqm+uud#z~9_t-Tlf`qDZ{UWl1~R
z50aqnVG@;}h#$Qui_u>0^L`Y;)yU@F<(p2}amMSj)Z`X5B>YhQAeLQN%39^wZQky^
zSawuEXY#z1b+_|?k!JxFuVu`pQPy?qm*%rdit=pP?_|dKevtlFgQM||tR|wv{aIHd
z2Z{dSxItTqP{!{ixX^z&&W>Cp765Yd&}=n8yzytv``-3sxu^&M6MiCBW|$6*X2uNl
z$|DFE{G3AwS=tF1?8AU_7R!pu(OwXpptQBpOdA=E;I<3}@I$-jOgYH|+>?==*;hw=(+gN4x^74;LU9r+T
zxn(hb*3s{$5F}C^_)*u{&kJdo7{3C2A~zZNO&(?#gbZaauFEe49ujn$Hr=@T?4piF
z)xPx4Iaqqv_cG&AT3~Z?^Ml+oTNisjhr(OU``(V@JU7106WFa%o*F~?8x;bf;oj5$E@iLzgmo!@(AdeulK0sq_PCSkj5}Bf}u2_&Us#diokXZKj9wgq|&|
zs0vIg886&;Fv($iPh*j1Dq%x<5DuQjmuIoIYtb^SrquyJd|+*pQFW#1)K)U7N#5fR
z^{a|y3(j*C9?^%e_ln`s4YR3s_%0<$u@#S949dFgHJ|R6hdVc(ERH!T6jeD|o4>6U
zt}c1Dr{i$*tL}I!wNr)3y-274Z)?!1v$U_PC3CK+_`#f~^p0+TC42IHVAH|dLjncG
zw^Y5y%Ca&DFk&>1s=8N%BU*>#$UZqR4?Z`=`(9m7#)pT>y37=Ix
ze`KX(BHSlcS{znmNKZmgJbHiWheo?!ZkwG+hm_piO2ye0cNcL|vHTOhMQIcrxA9dkZmMyI
z-~n0}Mw*;J1BqIumpDpyNm%*p>%Wh#P6!fcq}#!_hB{Hixuu%wWb=T$rwAOaY;Xw5?}*;?HTinmCb
zbtOoXYm93b!CPkRl}Vw7CxTH7>E5P|CS*Q(m_m=ZRl
z@MI`vAFu`qbtK3i4&eb;Iud9S9NTL@a4xY|$Ux}z_-s|}OqS%GpIOYPbn~xEDVB}t0BtSXi{aTHf
z#O3|6bVahpRMK2@2dA(f52Ae~Dhz+*HTMLRr*9+^mk4QI*%3%MjFdZ~4VjDLA`pU&
zOi^!-z$gX55`+rza2b!wL)6FWTk>@osRwvVFjRb-P9OKF
z`f4)PQ0-(k)>Q3dc5KnOPx?oJjnstxOJH~CKLpP6<`7BwoxsBXB`_rcWovy30gm8|
zDG@T5f(k&%vPT>5s`lLKYzZfz+(sRzaFi?+Wm3PPTTZp1M7#~m+jXNjsxOc$&8K3%
z!YZ73db89|GR&@t=;;;g4$G)e+Eom4_4@?;p~D%MIA9dAX58-BFhW`e_R9N?z+=UfFDh13&>9k5E@CfpVUYD^=o`LJwfyo%53adjef%w%
zU2U@G`-4tu8&~>(vCccIF03~B^xNMa9o(7965Gi-S^L)adUA47=iZu7MU#5=zo)H?qg%YY0UVM!K{a3EKJH8~#X;w5(xxO13>!QxZHr8dU%20^
zG6`ya;tBCt$iZW+aDur)!QiuSTdBOeBmsQg9>VRGpOxJo1B`6qea-msNo!zKtrUu|
zmnqERb;2gyXG}u`Szh-Dj8N>)rlGYpsZ=mLXpKns068tS^O?S?-_!qSwJe%jd}w7ahiMq!no!D@
z*PGPGl$pd!pP95k@=kgCfhAk$uof!ip$$LIUhxn6%q_{tnWm5B*}Q@Iv5^O<74O_%
zM`?a20)*z-R4l9Rw643!QXccS#NebSO01Q>4C{vDswk)m4W<<5yu-fwvHr)<)X~vc
zL@!RJC@lQt4I&onyo4SKW3_iPx{i4cfug~aUp+6ow^sEZ;k~mzK~6;qIG!|5#Zd(s
z9#&T^oXX#6sTd2;{0h|fuf73RkJ%^N1-_fReFwV5NZ+oQ?5ZBwpish%ybDPae9X9&
zE`%!akCp+&o5V6{9${*2n2<^!iurEVRv}ItAzAJuiV`XXZV-Y$&@Z{1*-DUUV5j(DwD*FAMuz@DCUwx#+}z$O0jF
zTO*sWK&yDmr8>5TuBuY9$|a^gdCGy7WUErePTIuCbYwzXoo|qGVue#sq*J1sdrL`f
zj1xE+n*?s#=i1JU5EaH^6&~T^h`fj6lUI92BQei6r`NNqORknjYm{G4rd&wYXM1Zb
zaH(kV{@U$1<^BO4m3KUsm)t)Sg64X0K#vG@F?T|cpXgXHVv(nGw1VJ7ZijCejL9Y`
zgoG9t@FmQlzHE3M4Gm2ywFGrYKB^1vOhM(f_l4p}i9H4uTSSpS}SL?d~Hb8Q{O{
zeWVqG=|$fw29{raXv-gY!T`PKhBESEoe%e5qR0Q=cK}E-W*9Pvk@Vd5KR+V)U#p!O
z{g3|hsSlAvPxi&T0`x>(*1J_^t)~v
z$MF^agP8!1aZwUWwTi0Jfe;b6_YS`#o
T2m~TWe=V>
zt6yA+iDZfLEUXueaIQIWr)L)?LMR{*CI|w$&v(PZ+Dcak8-(`1#@qk_x1XJ_hLp8F
zc+`VTPTNS6QTARnwX^`e`5P{d@*tP6+;l1ex0f5_Yg;VEh0>=?4>h8^61K>|8L_yW
z2B3Oj4Er!eUY%-i0Y83OEsnJN!=Puzc*M&yA!U0~u2xdSZw1dCHO}rrxs;K5KO+3v
zXn4eMpdV1_G*n-pXxs&^y#`z?vUUML2Z8)(t|4~6j#%B(2N#5b{s|2Py87ES7cJ6(!mn3%d6SfjJbKHi@7D4jhq`Rw*x
zdiXdi^)0o{eMG)1O@h;l!&k+^66MqMb{=~s-#3TL+=0(?ugbG!rX}xBslsk#d4eOU
zUN$;DWD*n45*_uem^*s(>hM$2q+9iT(eP-hjHJuYLh<$B>K9Qbmi$=s!YtHv$_t(}
zLR+)LaVtU{lsjHio}s31(&WF&5N(l;OVPwhVOm3R>s%meU64#c#=M%TG+>KOgZGn|
z3QBD_1`&Xw6T}23M`FLmpi<^}rJKq=<7}|AVKRbu`Z3Z2Wrgcor;i5p%(>9JN6zd1
z{MJ%};@I>gVru(J0qM&r=vAyfJl)&ai+nlL%z>hmZsU_0!j4L;(S?OAuEE7px*
z8V6ELW0+Cw2eQdNOheqnl!i=;)5VQ2rUIF-O)vmvgw|%_zfNhXdNOshPqI42hN+7y
zc>&4Usd~3qCH%uSXB#1_>5Pi@LPQk}i*`njw^q3s$rEQaevT#T@4ZR*Fx|JNj7m>FDqMe%1K9J9hhS4~DrSd$!NcqH9FIWtAs}
z19Io4A*zfIB5tj4&B2gdKhc}MgWqlkpwIa+1iLoGLp_V*%?_;DeQtf#+a^85aq1qA
z8reyeL3eJUUmU7bKvOBnvu{mYmHy`Z9=VI3I`8=SR8X>YRI_M#D$~jQ%!njjF*Eh_
z>C5`}zAuJOGT*Z-4S1>THIvUua0Rv2ngAroRV!mBx?}9wPT2fcD}SXRuczX3x5)C7
zD&8(i8r21GNbr8SnV+#LP+;!FN(yMUc#(
zcg4K?jQ+tlN)iV~2|H)yQ=$4U|HLSI=3`82@+p_}%T
z>=n5Yw|NR)lT>bJ3SI>Yu7DMNDQziY(vZ#L!oFc@@6~tJB1zN~3f7PN5_hUa`0sGX
zK^Q0$)bOI2-FrU_3_fsKb^e2n5fDt;-+YXaz#PK9WaIWQgm)T(7>nX}PhuiB@a2Mh
zImn3!ya{z`BbgilH{!chegR33^?KdIc6Oy8u^KRB)n9^Kll8Bt-_%*ifmdm
zF6S>=xlSZ_i&_o0?v1euD?
zKhVJJA5w7@h8iLtKg*gWoN-m;x}aX0stf8JZTI~=I{$1Vpt`9^_{&gL?6Xh$(axDb
zg;r;My$%PjNo%v#Am!LoDY$`6M>e}xS*nGVR-yHLLst22lTo8XgKUnHpB>&52AJ%y
z0=?9*)X6xO#&yK=u4npKv~`>P{9=ij
zn&(H0fi`tsD04U-8nvC}2s3kN3^%_T^G7^>w*isz?w}9zjKG06`+dzcpbGK<(k}uN
ziE&R`<{MlAN5CRDN*fiszlTrI@SUY)n5a!z4*o5X6DX{%Y9k_y#3oJUDs*!C*C|Pa%WQv6pTL
z$hd33L0&O@$iJp#S*{Qm$kQ^{vO
z^>zL;t&knk-ok0PaZ`qu(4&;BsNtPQWj|(_SI+ZSP$-#(k9x>{(C0v-p4-v^Fl6<8
zky|@hUvoowj)&T^nX32w0_}u$2+l1R_78B%@jjVGd}w<;O!C
z=M|7A+CrUE3ViPKlwvPVy|x8Tg?^x%qkj6*$X1%7Q+Tpcn*Qk-ss)#Z0Qy1;H5~hL7aBx>3ZlsTimHhFyI2~
zp*ZlKiAfv+!qm5qb2am$sN%^~iPQ&{bBC~Ndva3LiCB;-@%N0nEYhdzJeDG;PUl5m
zMp5nK_=yIAjHV`66}q@~8MVOsoSuqB*jZ1X9whLD>5t2ZsUknj91->DzR61UiMLoE
zXh?Kmo{_JNy+_+}%WW4}F`CIpwhd1fHkmA+g_>0XM_p0oPf6%Dbwc;XLx0Jrbt$b^
zJ2iQ(7u}G~6`1s-Vf~fVb#wf|4X~;3)`;ggiG?k<5@9UMp^Sx+wJrA`K`cvgM1PXN
z#jTSq_w74)snx1J);W@~XVxjdtYl?-p8dER+5=pqnb)(-ncFK-)faj3EC{iINA#PH
z5$AYA;7g)@knN|2c#Ti_yCX4?1wl%{3lQcGP?G7wU;2(Bzwy>=78b(E1*)Y>_0mY6
zFrtP?X<&%kI*o^e4_)E5*1QR%lgRWvDcd-FC>Qdi)=Ns26SXu9`#czj9=&WNG<~p<
zQd)ox-mN^Sddl
zSEg0B+*55Bp%~0bN~D6?whNsn-5ccJPZ|iDOy*_R_loC>j!$wJmVOjy6@E<+I66&V
zs8%hubLyhEF5L5L+Ms0r+C2G^2gq09ioJH&O<6Yx}kuHNIYAWEa
zHCBI8OK^>4pc%KX*Tv;z#%)MNb$1LegryzKM#$CU+hp&Ymq4F-S_l&3{cIaEDsP-l1)dxi
zZ8Sg47(VWQyp~mYhQv_zQ*~&x7qTHOj?(U{%9Y^Md7i3o46VIjECbWQ-dXJU0
z6u0w~_Y?LotY6EMn+M8KY$QF&Kj5+xxoD`3Q+KN;Wss(y86$U3h&DTPVt0;-VwP~w
z)h(B7ABDZ>TKaIG?LfG_CnHST09Y3D~ObH*y6Mq_ESC^cArbkRY@d~FG
z+c5Rz_RHC$u5hYN9N6!pf=h28o|5wX+;Tt2-cFm}ouAemHy?EzhsAzcAGvyJzShnV
ziApG&h?R#PY!D7_FJ);%C4T+ZR3AR4-m94-{ZYVArg91#+T;LmTEQFU%w2*@EhEUs
zX>gclberSwYiR}Wj?rh|M9Xt4oya%VNKv;45lr0amgK|}O|T})D0kcSK{!LiV1E#=OguT=2d0unum2|4;6e73m|0qk8Tq0@LKYd$3>lf6ie^H|Ym}D>
zITh7GjEaWr&gf^dzR^+r2|16`Sg3Zm4#*ClHsw4|W2Kr73G0JZ4u3MR`qe*@>mW1p
zxDwxlw)K{W<53}oNo%jPidp;Sx2e~)3W0xX6%;%a9rejL
zn0(6_xVhBrA|DQ)xE&`=YUQFn*J!0<)x=vlRDiQQF+5TpGV2$MK&+E70L8fngl~>E
zq{FD^83iW|6AC$q2=0P)5Lk+Q_?#kG$~JjvIvVJnrCl^7MJi_Ot5UHWohU-*?CiG6
zP>7l>n9l_B&o_WmNvPG+U2rrx@!uN7CDQ*ZPX_)HDfIsqDHygFnpri;n!?Hki%^NN
z+0}LwZ`U(vu$ird*xh3%88F%LtV^nk_L@nMY3au1$mDD$I}Y+&(Rvw7y4D|im+-YV
z%Qr#|zs=akXLFTVbAr-(DKgm|hl^vMM>EbwJF}hzRM*uBj8)ND3#HOjsH@cjnN%ES
z8r!{k`S0Sn3lwu19&psYi-)KtYDpgWg|r_iYz(Z-lsV@F)@e$%wa%4PWsBD>mbB}d
zipOi6-EB^6I2LWfMhyY_YUy9!2y!AsbHYQ3agYfb*<}p!bq6PONTR2;KLw3fwKD`_
zFLjUa-vJl@V17HCGI^XVa;~ffXf(`w1T^WCU})}~QRaV@=Jk@<73i+InOmV9q_UC!
zg7^Nf)>*%@$qxUuL+6aW1+M$|j!oP)GRj$Hwrma}Si=ooxXveSpMNW6-uPiwYc
zdGz9&?8tctfK9%YQaVG-EtLEp6R4`k#|rRFlQ{T070U(KOg8-J0tTbffowM8!hq~Y
zy1q6xt~Ei?xXsOx(Q@~=ZxZ#`=ZX1I$KXElv6U}1EPPnFo-8_M#Px{ZDry#UO}F0S
z@PmcKC@dRpiA&UJxo&BuqVuA~y`zcuM^xP{*reE`#rVlPuXynp3kC-N>9?!}tc~mX
z^>!VK4OSEn9g}rPkRu0S69Aj8n1vjsnbn)L4#rP97+L8^?G2AHcTtA4GYQeCRY_W%
zjt!maz)<(Hs3;DgqNs=pwO!R0LuemgRI(TU(dhDV*ZwY^g*?HM;D#Y<{AU}77>!5H
zNuf-t@i03h@7_`G1#3u{=51T~5W2qPM6I*6S9ZiUe(+48L^Dnj0LUkTH%34cT+w#T
zu|E_N>7Zfk-Bq+&DHGXfC@J8!+FaSLugigGo!kL9>i!#rcrxp@$a
zLsx!+QIj~z{7y-7^(G)9H;&!YNI4r)R8ibRA=7jw$U-SInN*ERzn9yHyrEEA?JQs0
z>@2VKUYMC5lzM$3KLXnO
zKF_Ib*HH?^pT`F)mJjOQ>grF8&?HEDl3-cDs4AWj?NR^WEFb7N>{ZXAptXCa0`YU|
zi;qo|;MA6_z&!68QS3dT1NqOaJy~*Vn&4#?EI$7F4boX~%=?Yc^po+5e_T=<+U#Ji
z>9%jhuQ#now55r!!>yqV#~l)fh+V7?ZAM4RXO;>y;diC!OBYCLU(Hn}ba%p=@i>XP
zzX`v!G7~swltLJyzAXBd(`WkgL=O+5tEK+709VT1|M4QQl_wDqrop&13r_Uh*YLvl
zN%?0cX!-LObInzb3IbtVYPSM6-0iLlp@>!L;Vb4GR4t*lzObCWxUN)Q!GAL+Ibw^3
z0zg4y=NV)8teu{j{?y)_$Qk)mPX|ms6EP73pFD8O&0N!HrM5QW)Ip*t59g=pB^iRq
zy(1>#piL=o)#(tNtz5g1xqd2?6!rh4#C!Rl12HZ{=CXAK=G*J#u%Ag-`5(wEA}_AgGoC0(|@ebQU3-
zHX2wo-*XZL7MUapRd`uy(943!m2rawak}q;y)tY}jU?U$vc|3cXis|EPGUju3Lo0}
z5N=Owpac(eRkAkqt1%ALbV9%(^eBpm`w$udTbJtYk+2r#cJ?}01Gr2U6xjk76kr=L
zke8RolqZlk>$=I0`X~1zN+H*GFABs8qyg=sWI#NlgZ{1-5M6Y}S8eq4OuySMG6cJw>6k#JPA^-Twv0WsUXE(ZuzT3HTR2mu1htar{{wU3ao0_}l5}*
z69tL(Md5e*;rHqwwbp$@2OQtjJpcdz
diff --git a/lucene/core/src/test/org/apache/lucene/index/index.40.nocfs.zip b/lucene/core/src/test/org/apache/lucene/index/index.40.nocfs.zip
index 460feb068efb325a46514ae2edad53038928dc26..1b50722c79cd199dd076e7c68e8a6f75ae045c63 100644
GIT binary patch
literal 17101
zcmeI3c|4SF_s3_9-H<(7kzHgNQqnR;$QqI@%b0AHWUENaWX}>I5?ND3LJ}g|ShE!E
zlqKm)qS8W=-#rt)_dUL4Jdda6^}Jp`T)g<>ea>~=*LAMXea>O1i=gI&!C>?-J==Te
zOa3W|g1~P=q`wHjY~^hgWbGYXcAGH35UE^`cM`{8ZzdR=dI$l7k-kmM%*?daKf(
zWRM_cm)#^bsewcy!_h4?grv_cFhhs|(P81*ECB9H1B21Rhz4XG?Y(}7zgd|qBHOp?
zV$#L<{Ww;uXxrF}{c#BwV~j_l&7&J()Vblva9TZN;nqyS!QvKOgK7EOht+JZQTf+o
z@G^0-$80rtA-_c>Faf`r$$!7*exqv`#{!J(ea66n@8!`C%h>}zwnQTX354xDjt)_J
zh-WP48t=7t>dP3kwU{*Ow&>HS5Zv-N5}bkI4Geo=cmcx&7{`Hu1BQqEMryKoUcW@g
zh6d*80gU;1dM`T9p?{iZLVUDkH69jfc#JCmrxeuTE2I=79lMn-Eul8;6N0`qnjv!=
zoXmc2dFzo_z;+$g-}YS(E^2>Z)_yN>h5HlY<7``C)B+<-lRjeEK4Qple!P|VGyFs0
z{80S&{^*wSPq^FP)4K`m6-x4Ta7F6)!v2n;q83I?#FZA#dNl1?zQ&8@meG#Y1U-GN
zLVURWm22k&?%!9!rR5duOURB?c5T&aZ7W-jbV||NPyGw9K$~
zevi50`LgKjBAz{99&yx-PN$3fCJ`cR%Kbqmxh}>g%x`6NhStu{!%f=ai7cVLOif>7
z62{8y?lwJt^0FX7xApB?v#wFoj0Z>f_dIbe+K`ufN7IO46O+f$YHb^kZhDZpUzC2V
zuFEJYq>;;LTU&&f&$!8@-SQWXzIhXKd*^i9xtF8WFEirrZhY#y8A&tw!=d?=(LX}?Tmcvxwrf9)tw*9lW4|W
zg^sP_J9eTk!2f;H$|<#|>ubLZwD+x-oN3|Bs_`j-jT`#YmV}Mlnd~Z))E66U$EV!=F`Lyqn2ZXGz^P|RdtxT)_qqFWpVC8~<_JwW+x)Bt8iBC~4y~{Sw(HbCZdtE=
zZi}9x!GTv}BdY}22Zs3s`}qd=1;&Qn^z#7`YV6(V<_jq%H+a*wy&zOGGeoPtP3(=v~WoORsUwj!v5DEu6rMQVbDYqyY_*8
z>a@)BwXsu|3qR|$^yC!tsXHBYsq;<<8}E^N$Z{wQcF#%V!2<~&hkd2AP2%d-R`KNm
zha^?T>kUMdKD|G<1%1_>`93OpNYcG(H070>(t`;F@3L)?!!y};MEzRczuyMkquf2N
zA!~ub
z3a)Nw*CT6t_4*#aG0!sTCvT=3s)u}Le2|Rk@DJ^y^op1^!eh%eyo9F9aOoy5zD@75
zv0i-qiuGua1SzV?d1i7EW>f)N$xy}`Hg1e-svcK&(42ETL`j;>8(c0toU8$BlP
zC7y|Mp8fhF=WGUD4X@TC3D&@BRD;)j@jGl@oq^VURqr#ujj%Yk{$Ql_6!B*Jy^?h|nzj1Kw}6*(Wx6v*d7VWva~sZ6Qx9c9tUsHmNmxM6*yWX5Y)@$-|gY&FggJUKa{43A7!HNexWz8opw
z@@%^_!8R_k*&dFNBfQeCdao|W$K6=tvdxIGxWdivvYqlR-_`ug`D5V;Lp#6UH2CY7
zO`-H#ZJYGlI|k&6@sCd%r@p}_(>;ehr#=S%OC$9FT8=GN<3}f3%C|G0R5icf#C*gs
zVnQ)Gn85&Z%wS+_ph=NhqAp7KV2H{qqw_0P;P0%q2>nJc_5P`(;H}366Ca!5o0Ok@
z&l}8fKC*v%e>s8XOc;B-XmZTcXm6YTLIRgZN8>lgkQPf-jd7$wQk_(ACpgF5{>8;)eIyo2*Jb9Fa!<-gZpolw^-J@KahRFD2gqb)ne&cJz2qOWUm
zQXlH~Fb|e1xox0?6!h8}N&mfxEj_+3=b1>UgSYKP;dI9&gio0$_@EN_XdD4H?J!dWEazw?FO~-$a)FG4JNcyYY2zs45imS$P3|*sTa)d
z&RDBooLbN8Zh2nfwYyNR$&OmOJ;jw{PLrcIOG_jW>n(*hy=NjJ84&Tcn6Zc8_
zip4Pfrq@jy%S!^O%uV0Ba2$NuD5BN$9NU$$!M8&)NegYd^+jduiuN13Jg&zB!cw=(
z;__}A*O=zVx?4}>l}g%&Uo^hYTfh6w}o|e8RuR;prdivSAb!NS<@z~AzLp<
z?T;NX8DK5j8o~65`|=e;#Fw#cd=X!;1>@7(x~3zT}k&viz0Cq%C6~WHHSQ-KPt*
zrGo6A(w0O^|F129?Mc$~#MM|_)%+Q4Idc0%A4FTCmP1=={tswN;zSpJ3jaB63E2N%
z*Om+NlBhqUEtis)gt+wIkd@RTc0nV6tTddHl@N~>k(H#k`{4y?r~V;<8W>%R-xu`m
zj;)zZ**@9gwD8F~u>_x?x;~8rv22Ys*&3yFhJjQ-F(O^%ATRI6`Feod)laqEaXwAi
zK-ECDQP<(5(q1^-N(8lsq?G$^2WN5*5srVv*?RvHPPoRW1C|w
zTA~weW1EeWEuyPy#i%;l>#8`|g@qevXp*kv0(or}DH)$LrP)b7lRjl<>Y5S~u-0y>
zrH%@+b^%!{Z#ke)SQN@0g~FjwjwqBJ3I+USFAC*^D#W4+2XFz1QFZ#_k3O2k)GVo5
z8bYy>0qp*U{j54v7;TKC6ulU8n!Ql0%z;n@0wHpMmMN+>Is+!kcmdyMEnHP)Roz%+
zQoAN3pL19vO)>m0TzzmtDHo4jftVkGTLTxirl9WJQ}!J$+M>mr?0fjmtn(+ty*BQ{
zqj*);3#959tm2jot3|_`d8VeKsmRGe75y=4S;Y;9e0O;H%L
zWLsn1r?5+5Vfs-+d`_^HcSJiy@6v=@waDo85b%X$???N@nOR6eic@TszfbCRc6eqN<%zRRbI(L1=8~(On6NL!WpjV@lcwroHM$ts`4+DY)l_m
zRiYMEm9{{(lCy3;TS-+x;Q6xluF6v{!ya|*m~|c9E#IX+$9!{F%Hy_64)*MI8(lG4
z>!f(cLSu~)%DDOZIi9$z5y_!gxc0eIVu=Hf)uk^z;l43?no3Gix_GE%pF8eEaR<+x
zdxL1cpiXMZ<~I%clb=3r`l7sPjpC#2m5HL3uL-g_dKK8t+t;oKW!zLXhxNr?nkhEl
zb99ep(v4egk55eqMWo0XRb@TVsF(8IVvsj{U+XJJ+Li!}XRDOp-2rO9mv6GwoY0cb
zcD~Js@EFr-eR+6h<6z_~-?QIn7m<{xKPD-Omi}K-&Pht*YAh~h{*0vjYwJ|QdBCyG
zxeBb}a!AVhKPf2}rY651ZUzT%Y2N3!t0AQcJN`=6JKZ%XcO2i6Y(
zs3B5?R&jadA{rlo#X9<7l63O-@3qG{o@NUNS|q}~8E}u-w6LD6WJ~<8+e1MUa3>5H
z3*Gs*rJ_(3l1gYbf`cA!ksBV~NS{WXi^x4oLtTZ#TIBPH2&D3o=lN4EDRor?YePIZ
z3msZSDFPL2*G@ER{m};1woUp%?95wCbh<;-MMF8=vZ8bzWgWaM>=zO$wlVcey+ueS
zL9IB)t#^l9g_hfp;Ae)7<2+VT$NUjyj618Izcmzb$I7&S9rt!LO?mxUt~T(a=9>?L
zwaFEou3tyKv}H(CT&GJ^p5&0Xj<*iGbi!;R{n=kjsll
zzlii{xd>6gZ5lOSq4UqIWM;gO(kddAn>Tm^$5(HCeCUi_%4FLPo*z#uUkBv95!)5-
zG+K7eH;>U2k=uoR0l!C;b4G~;*}RzuFpf4o@D!NPFX2`E;Vx$U5@6??c!IIN5}UVhY}yq-LdA9o8m3Q>QG
zOHqHF^Xe}oumhL$p*{;0G^hU9u7xXM>Zy0vuUe2(-0yksII&jk2@Fz!e9q{S)gKj4
zI+qaova=LD){KcPK-|_;$>|3y%;yvpDOG`*)vE-e8s}$tBsQbL$pk4POzzScHm0Q={3YE1tncyH`i}
zhxrZ0I;gZ4XAc6k>K)3oxT7xxcGUI@!wWwS{m4A-eezOjn9bVs#E?7t(s$d8?r?a^
zWm~@c;QCm{+g=(8jgv!%7;lX~aR_(~q!jz;XdT>3X|=T~+6L2Ed)J+}w7%j#aTcp|
zVY-rFuw6}k&(|xb^q%x#u~U1VR31Ks-CW~5b24`1gqAT=>jtR&3qC&WPpnoGdy`0+
zEyZP$e*w;9k3t%wC)Nm9*+&hs^n3Xiso+|JX4QBz79PPswU()RRg~mkotZkknYM9i
zx%M~2xOcecKJ?LbbM
z&qvE#{!j@$AuOzUWccHc>AuIk?m8K`qgz+sO-hbDbTw7zn9z^u&DF?rsDfK@9N7Ah
z^yC~&P8!V$U(r3+55d>G;ulg7b>P}2&JkU4=ysG9?oeUr^^p1i(Wl1Mr9wG1-O6pg
z7u06j&zk&Dys7^7{-&a-RBN)Ty%o|wc`L9E?%)+)T!#FM!jHVSuJzrEWNE;%?3{I_
ze^=S*2_Rd}oH>((WhZ;!mXn`*L;y3y?M2#JL{p1T^tXD8%tVu^pqD9t_XnByS0$F7
zz-*^ry}((1^l?(BVDvwA3ex=DDR`kkJPC;f5<`$a(bPsOL1JhlPM#zJl$KE>k$3UE
zj?H&62Mjs?qp?L4$8T~|co@TCsj0)H)Kr2>0tW{NFE8(T9u=MhR|N$Wxr@;U`%2dX
z?y3F4#eX9Y!tw>}(BfLX2Cb}!5K-)0C!W7i#D5dn^SH4=V_+px*gsmV%d(Thv%0F?r3;6aB+
zmQv@1E56Yk3F+#+^~tn)zn$+2UF8R{t=kp(AJ{%m+2voxIU+Dv?K{5p!jqKY$$hIY
zW9lsOOik^!sK2vxe=cAio7huQc0wzES8{l5!^%c{ChtzOg29OL!?7*QLDTAoh1UoB
z)9)Cs>vH;W_xo@Zlg+j5W~U#&*dbkRB@wvp>hbs1BcWFwCRTA?Xq=d?tLvCFTJ84T
z?($KuR7(rKc0H^``r1$lpKN70Ggp$+&a3d19+=mI16tn-k?^Ieh~g
z2|ZVs_bhJ2J(M}S*q#nf2OV6!JZ%+!*YiZ&<84Ms7n2h∋p?VVlt{j5T^6Xa(l7
zu7)a_td5}@0>Wm3siH)8m9w$9?-l45H?Olp$DOz#OZH%DEnYlj0vsUrog;znw>c+d
zog7?$=LV_MoR%IN^Z)?Dsa7L?)ws?BfPa1zohJVLyN#U&MrvaxYhfp+LA%t!G&>UQ
zXS}D31?>a}v$TPIN$P(G0yZ!$Lb(!Pp2|5_Z1dfHEGm
zDHY74q66`OfJqq-+I|Y=Ig@y#I7t}_T22R}g6WB_{glF#v7pVEVAh8@79e&hV?kRP
z!K@jES-&VgcF@j4Fe`}>V9oX+Qsx)5xev?=1VSQ-MX_JM4hJ;IpDfD50klC5?AJ3A
zYuUqrJ2TNmFm?&y0NSJm_7=fHj$J}Hfc7wfv8t>Kv4G~H?3qIwj=~3$RNF2ha)<7#jfOl%U=&Asj%95nybT
z=t3;T;Q(4X0Ap3eNZ2KY187wM%*zAwC=LhE{6Cn-DNgin3E=>mDF>t0OOQ}Y2nWzy
zIGB|($D%kKK=a99)>X+_zbHO-(CjdnB_;*1W^=`qg#&2%7tD&9V^QoEG|vlWjnA?u
z4+qffESPl$$W+@d%+*phU1%N_jQz3zyM%KK&7Fd=2|yYj>g^I<@}VhFFqT(_gk9pf
zh2}xQJTEYh;&Tg4e}Z|fB;FFvEi`KgM)j=&%8#HNB4rO6ng9gz9D$=vp@C0g9`QD@7z6u19>sa{
literal 18257
zcmeI3c{tSV_s3@}(^!&yOURZzhG>#~OCm)65=yd5B}?{gkfoG0
zOJs>GB_Y4hOv`s>`j+uLJ*?@nsqyL8mWF}%JDr{wKudhb}B2@RPKZ+XXJjg&`qDev!2>r+E#>R&1>NI>L
zNi&|?JgbR&MVXwOJWLKrT$}U(3DUwcz%|8Da~FVn5raSw5XykCjg{-~@I{q3IfaLH
z6Os~QPgzhNk2H%;7>S8bh|-yfG>U8l5#0|X4})ls7s_O?jh7(RHP=NRo|8XukHEkB
zHWL{wb(D3k
z2H`7;>y0h#T}K2p+mL#V>c}I+@LI?G{k3+$@BoGtFkFFQ4~z@IumFa$$bKT6dFCW<
zqml#W=?sh?^Yqwso|FG=p7F7fCRM(mD_ZC20xZM>JH6P&qXeR5w%v-axwS+HZH*+!
zkOkw|Hyjdiz5&>-BlwSfhclbnANywiH7dd*@v$*xNDvXrOw)=dca|qNdDu?h<2z{|$T5ZXj%&v%+e2{!S#3wH)C|~@0Rs86O-JY7K2uAp0I~*Kl{yJg
zq%!O7QLY|3+YIYZt~fVG>oz>v!gN5nXRk+mO3`Z~f9;Tu6d^C49Zg}9;6B4J2~XT-
z8hES3L`T}kXKI*){JI(4iFd&wIh2+Y{Q}Qw^1U=yn>8g|#!t_cIMW&~6kUWp`lNMo
z?p9@pz2uUNmBE{`qfVB`lBZnWPuxhJEVPd*XL>;+3Xd9D{bJ=7!%@a5gz>zD-V@UZKD7P%u&wb~to#Fxs(5
zu*;*cF=hp+eJ1AIC4TeJp(-t{Do)vP3W_3dlg
zAcxG5NMiUd!@W7WR_C9VT2(nrSMHpJ!A*QdMOw^LqL5bW2b;Xgq3r`~8bv2Qs2_RR
zT2T-&d2eF(yS?gWIm4E{W*)=uqTm`qDKNUGvNxsCQrYv=oY%7ma28_U@+H|$U{S~c
zIq>I2;d*lOjW#tlIybY6je2Z~nQer5Y$way$RtZOXd5!@qny5lOh!|dXBH0xyi&&<
z?>SjFtQ^msrL47&07$B7@H0$I`+wxMy?2&s`eQtsvViO-+^BBtEp7?k#z08q4BF=l
z7Te=uVZlMeKo}JW$rB20y_u9^7@1&L9eLd(x;fey8JTDn-K=xdII^mSo1m+`?y02}
z7e@mzaZ=`eV0*&(1U*9@XC-;Ac$S9LHO0pxOdX}*0tt+db>y}5qB?v#u5|==#B~I8
zM0bRBBz5?8M21pPJYryf**wAYih94?V1SkuD@}k!pq3!z6*`3id0sbY9r)IMV_g=x
z!!RScDHmE_B{0o1UurO9oKyvp$xI<1kfG&6l4GI7ASQ!E<~)T)uyRRoQYVsxgx|kO
z9MXEF)<)5yh_e8G?Pbz&eip})K*F$$8Uo$ZsUYh(5&0v9l)ld-C>39_JcZCAY4qq@;&F&B?N)h%CWSnxZPx`(rZrB&<$xNoSV;c=wmnfon)B<5O
zMHo_$FZ#}sB!Ae#+T5fbLTfu_alW>#u8yZsJY38-)HHIEdX7Lv*oHpyjD?&r&f;i(
z5d|>=veYUt{@dbk{9TqppLXLvEsn{F5oX{Bn0t4rIc~sD(bwJvqGFS2sey1(fryjD
z$Q-9$bv%?WRI|J43p9vmLY#%VEY|T*@EL!#yHZN@=fs4~`&&H1sc<~i*kf{66!0<#
z7^GkGse|(wVM`n9-?@l7?|lr~Jfc2D!^)#;sUc9Pe`jaXJmfCyK9q@5kBK-SjPIVt
zZiRYA{d#5|LL8&dt$M{d0E1z;MqPeubQv(}jZztcz%fxrPWjY`a2ulzouk|pJwBG#
z9mF_RN*<1|z6LDl0=ty>tIlva`8xyF5ku-VB9sk5okyl~y?UWDkl+*O-eFYY)(csiqGV
zwCbqd*FRKW^<27zsonYdI}7sggHF-*pVlSf8J4w4p*yqVAOI6t7-L>$fEaU^wJ
zu$Mo2HNEh|VdPMD3A3WDyM3KUeCXm3UpK|sP*964XGg~#PwSJ9Ax%7rrpIH;SkCgo
z7wa`S#h2cnk(RsTwEY<@Ym(RL>1@iVqj<+BQIAsDh^dXNCtQBW_wQw~Wf=XCLoh!O
z9!Fq&lP9ijR=roLFn4z};Ami|$IK6uWl%o!e<)okt<6VZ~Ac?}SGPny5JJ
zJW+rMV8hk9(Y7=?Iz=fEBgO
zhlI`U?~T^dDiqESkiud-B)uvxQu#Bf<=^mY@Jpkh={qoI9=JZV-+84aF(k(B
z+wYHi?F_M_;o9mJZwf@C3N!A%l@QqS7bEW9R3WLB3CZEpa+8%>y&whALEi--cG@l*
z-XXAp=(79@E9Dh#n&ut8TK9#uyomxVqJkD^QrFD1ydeP=m83X(=I-l;?kvEu%ip!R
z=pWGL8i9X7oKyTloEz`J73WWC9iKg_wF8C+Fsy*#3JiN-TmXgzFq}mm;fiy8RZAv0
zV4^?7IclPtKZQ-i`946LU+!GpEgrQex`O;xyydMW7ig;vG{Xls1^y(?0sH^+;`|;v
zIhQaH)F0wp^bd)1_NS0qN_x(C!BdH