MINOR: Remove unused IteratorTemplate (#5903)

There seems to be no reason to keep this around since it is not used outside
of testing and AbstractIterator is basically the same thing.

Reviewers: Ismael Juma <ismael@juma.me.uk>
This commit is contained in:
Jason Gustafson 2018-11-12 13:42:29 -08:00 committed by Ismael Juma
parent 1c1e5ee979
commit 6c2e7005ba
3 changed files with 4 additions and 146 deletions

View File

@ -1,85 +0,0 @@
/**
* 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.
*/
package kafka.utils
class State
object DONE extends State
object READY extends State
object NOT_READY extends State
object FAILED extends State
/**
* Transliteration of the iterator template in google collections. To implement an iterator
* override makeNext and call allDone() when there is no more items
*/
abstract class IteratorTemplate[T] extends Iterator[T] with java.util.Iterator[T] {
private var state: State = NOT_READY
private var nextItem = null.asInstanceOf[T]
def next(): T = {
if(!hasNext())
throw new NoSuchElementException()
state = NOT_READY
if(nextItem == null)
throw new IllegalStateException("Expected item but none found.")
nextItem
}
def peek(): T = {
if(!hasNext)
throw new NoSuchElementException()
nextItem
}
def hasNext: Boolean = {
if(state == FAILED)
throw new IllegalStateException("Iterator is in failed state")
state match {
case DONE => false
case READY => true
case _ => maybeComputeNext()
}
}
protected def makeNext(): T
def maybeComputeNext(): Boolean = {
state = FAILED
nextItem = makeNext()
if(state == DONE) {
false
} else {
state = READY
true
}
}
protected def allDone(): T = {
state = DONE
null.asInstanceOf[T]
}
override def remove =
throw new UnsupportedOperationException("Removal not supported")
protected def resetState() {
state = NOT_READY
}
}

View File

@ -32,7 +32,7 @@ import org.apache.kafka.clients.consumer.{ConsumerConfig, KafkaConsumer}
import org.apache.kafka.clients.producer.{KafkaProducer, ProducerConfig, ProducerRecord}
import org.apache.kafka.common.config.TopicConfig
import org.apache.kafka.common.serialization.{ByteArraySerializer, StringDeserializer}
import org.apache.kafka.common.utils.Utils
import org.apache.kafka.common.utils.{AbstractIterator, Utils}
import scala.collection.JavaConverters._
@ -199,8 +199,8 @@ object LogCompactionTester {
}
}
def valuesIterator(reader: BufferedReader) = {
new IteratorTemplate[TestRecord] {
def valuesIterator(reader: BufferedReader): Iterator[TestRecord] = {
new AbstractIterator[TestRecord] {
def makeNext(): TestRecord = {
var next = readNext(reader)
while (next != null && next.delete)
@ -210,7 +210,7 @@ object LogCompactionTester {
else
next
}
}
}.asScala
}
def readNext(reader: BufferedReader): TestRecord = {

View File

@ -1,57 +0,0 @@
/**
* 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.
*/
package kafka.utils
import org.junit.Assert._
import org.scalatest.Assertions
import org.junit.Test
class IteratorTemplateTest extends Assertions {
val lst = 0 until 10
val iterator = new IteratorTemplate[Int]() {
var i = 0
override def makeNext() = {
if(i >= lst.size) {
allDone()
} else {
val item = lst(i)
i += 1
item
}
}
}
@Test
def testIterator() {
for(i <- 0 until 10) {
assertEquals("We should have an item to read.", true, iterator.hasNext)
assertEquals("Checking again shouldn't change anything.", true, iterator.hasNext)
assertEquals("Peeking at the item should show the right thing.", i, iterator.peek)
assertEquals("Peeking again shouldn't change anything", i, iterator.peek)
assertEquals("Getting the item should give the right thing.", i, iterator.next)
}
assertEquals("All gone!", false, iterator.hasNext)
intercept[NoSuchElementException] {
iterator.peek
}
intercept[NoSuchElementException] {
iterator.next
}
}
}