2828import java .util .ArrayList ;
2929import java .util .Date ;
3030import java .util .Locale ;
31+ import org .apache .lucene .benchmark .Constants ;
3132import org .apache .lucene .benchmark .byTask .utils .Config ;
3233
3334/**
@@ -50,8 +51,8 @@ private static final class DateFormatInfo {
5051 private ThreadLocal <DateFormatInfo > dateFormat = new ThreadLocal <>();
5152 private Path dataDir = null ;
5253 private ArrayList <Path > inputFiles = new ArrayList <>();
53- private int nextFile = 0 ;
54- private int iteration = 0 ;
54+ private int [] docCountArr ;
55+ private volatile boolean docCountArrCreated ;
5556
5657 @ Override
5758 public void setConfig (Config config ) {
@@ -100,21 +101,35 @@ public void close() throws IOException {
100101
101102 @ Override
102103 public DocData getNextDocData (DocData docData ) throws NoMoreDataException , IOException {
103- Path f = null ;
104- String name = null ;
105- synchronized (this ) {
106- if (nextFile >= inputFiles .size ()) {
107- // exhausted files, start a new round, unless forever set to false.
108- if (!forever ) {
109- throw new NoMoreDataException ();
110- }
111- nextFile = 0 ;
112- iteration ++;
113- }
114- f = inputFiles .get (nextFile ++);
115- name = f .toRealPath () + "_" + iteration ;
104+ if (docCountArrCreated == false ) {
105+ docCountArrInit ();
116106 }
117107
108+ int threadIndexSize = Thread .currentThread ().getName ().length ();
109+ int parallelTaskThreadSize = Constants .PARALLEL_TASK_THREAD_NAME_PREFIX .length ();
110+
111+ // Extract ThreadIndex from unique ThreadName which is set with '"ParallelTaskThread-"+index',
112+ // in TaskSequence.java's doParallelTasks()
113+ int threadIndex =
114+ Integer .parseInt (
115+ Thread .currentThread ()
116+ .getName ()
117+ .substring (parallelTaskThreadSize + 1 , threadIndexSize ));
118+
119+ assert (threadIndex >= 0 && threadIndex < docCountArr .length )
120+ : "Please check threadIndex or docCountArr length" ;
121+ int stride = threadIndex + docCountArr [threadIndex ] * docCountArr .length ;
122+ int inFileSize = inputFiles .size ();
123+
124+ // Modulo Operator covers all three possible senarios i.e. 1. If inputFiles.size() < Num Of
125+ // Threads 2.inputFiles.size() == Num Of Threads 3.inputFiles.size() > Num Of Threads
126+ int fileIndex = stride % inFileSize ;
127+ int iteration = stride / inFileSize ;
128+ docCountArr [threadIndex ]++;
129+
130+ Path f = inputFiles .get (fileIndex );
131+ String name = f .toRealPath () + "_" + iteration ;
132+
118133 try (BufferedReader reader = Files .newBufferedReader (f , StandardCharsets .UTF_8 )) {
119134 // First line is the date, 3rd is the title, rest is body
120135 String dateStr = reader .readLine ();
@@ -143,7 +158,12 @@ public DocData getNextDocData(DocData docData) throws NoMoreDataException, IOExc
143158 @ Override
144159 public synchronized void resetInputs () throws IOException {
145160 super .resetInputs ();
146- nextFile = 0 ;
147- iteration = 0 ;
161+ }
162+
163+ private synchronized void docCountArrInit () {
164+ if (docCountArrCreated == false ) {
165+ docCountArr = new int [getConfig ().getNumThreads ()];
166+ docCountArrCreated = true ;
167+ }
148168 }
149169}
0 commit comments