|
18 | 18 | package org.apache.spark.util |
19 | 19 |
|
20 | 20 | import java.io._ |
| 21 | +import java.util.concurrent.CountDownLatch |
21 | 22 |
|
22 | 23 | import scala.collection.mutable.HashSet |
23 | 24 | import scala.reflect._ |
24 | 25 |
|
25 | | -import org.scalatest.BeforeAndAfter |
26 | | - |
27 | 26 | import com.google.common.base.Charsets.UTF_8 |
28 | 27 | import com.google.common.io.Files |
| 28 | +import org.apache.log4j.{Appender, Level, Logger} |
| 29 | +import org.apache.log4j.spi.LoggingEvent |
| 30 | +import org.mockito.ArgumentCaptor |
| 31 | +import org.mockito.Mockito.{atLeast, mock, verify} |
| 32 | +import org.scalatest.BeforeAndAfter |
29 | 33 |
|
30 | 34 | import org.apache.spark.{Logging, SparkConf, SparkFunSuite} |
31 | 35 | import org.apache.spark.util.logging.{RollingFileAppender, SizeBasedRollingPolicy, TimeBasedRollingPolicy, FileAppender} |
@@ -189,6 +193,67 @@ class FileAppenderSuite extends SparkFunSuite with BeforeAndAfter with Logging { |
189 | 193 | testAppenderSelection[FileAppender, Any](rollingStrategy("xyz")) |
190 | 194 | } |
191 | 195 |
|
| 196 | + test("file appender async close stream abruptly") { |
| 197 | + // Test FileAppender reaction to closing InputStream using a mock logging appender |
| 198 | + val mockAppender = mock(classOf[Appender]) |
| 199 | + val loggingEventCaptor = new ArgumentCaptor[LoggingEvent] |
| 200 | + |
| 201 | + // Make sure only logging errors |
| 202 | + val logger = Logger.getRootLogger |
| 203 | + logger.setLevel(Level.ERROR) |
| 204 | + logger.addAppender(mockAppender) |
| 205 | + |
| 206 | + val testOutputStream = new PipedOutputStream() |
| 207 | + val testInputStream = new PipedInputStream(testOutputStream) |
| 208 | + |
| 209 | + // Close the stream before appender tries to read will cause an IOException |
| 210 | + testInputStream.close() |
| 211 | + testOutputStream.close() |
| 212 | + val appender = FileAppender(testInputStream, testFile, new SparkConf) |
| 213 | + |
| 214 | + appender.awaitTermination() |
| 215 | + |
| 216 | + // If InputStream was closed without first stopping the appender, an exception will be logged |
| 217 | + verify(mockAppender, atLeast(1)).doAppend(loggingEventCaptor.capture) |
| 218 | + val loggingEvent = loggingEventCaptor.getValue |
| 219 | + assert(loggingEvent.getThrowableInformation !== null) |
| 220 | + assert(loggingEvent.getThrowableInformation.getThrowable.isInstanceOf[IOException]) |
| 221 | + } |
| 222 | + |
| 223 | + test("file appender async close stream gracefully") { |
| 224 | + // Test FileAppender reaction to closing InputStream using a mock logging appender |
| 225 | + val mockAppender = mock(classOf[Appender]) |
| 226 | + val loggingEventCaptor = new ArgumentCaptor[LoggingEvent] |
| 227 | + |
| 228 | + // Make sure only logging errors |
| 229 | + val logger = Logger.getRootLogger |
| 230 | + logger.setLevel(Level.ERROR) |
| 231 | + logger.addAppender(mockAppender) |
| 232 | + |
| 233 | + val testOutputStream = new PipedOutputStream() |
| 234 | + val testInputStream = new PipedInputStream(testOutputStream) with LatchedInputStream |
| 235 | + |
| 236 | + // Close the stream before appender tries to read will cause an IOException |
| 237 | + testInputStream.close() |
| 238 | + testOutputStream.close() |
| 239 | + val appender = FileAppender(testInputStream, testFile, new SparkConf) |
| 240 | + |
| 241 | + // Stop the appender before an IOException is called during read |
| 242 | + testInputStream.latchReadStarted.await() |
| 243 | + appender.stop() |
| 244 | + testInputStream.latchReadProceed.countDown() |
| 245 | + |
| 246 | + appender.awaitTermination() |
| 247 | + |
| 248 | + // Make sure no IOException errors have been logged as a result of appender closing gracefully |
| 249 | + verify(mockAppender, atLeast(0)).doAppend(loggingEventCaptor.capture) |
| 250 | + import scala.collection.JavaConverters._ |
| 251 | + loggingEventCaptor.getAllValues.asScala.foreach { loggingEvent => |
| 252 | + assert(loggingEvent.getThrowableInformation === null |
| 253 | + || !loggingEvent.getThrowableInformation.getThrowable.isInstanceOf[IOException]) |
| 254 | + } |
| 255 | + } |
| 256 | + |
192 | 257 | /** |
193 | 258 | * Run the rolling file appender with data and see whether all the data was written correctly |
194 | 259 | * across rolled over files. |
@@ -229,4 +294,15 @@ class FileAppenderSuite extends SparkFunSuite with BeforeAndAfter with Logging { |
229 | 294 | file.getName.startsWith(testFile.getName) |
230 | 295 | }.foreach { _.delete() } |
231 | 296 | } |
| 297 | + |
| 298 | + /** Used to synchronize when read is called on a stream */ |
| 299 | + private trait LatchedInputStream extends PipedInputStream { |
| 300 | + val latchReadStarted = new CountDownLatch(1) |
| 301 | + val latchReadProceed = new CountDownLatch(1) |
| 302 | + abstract override def read(): Int = { |
| 303 | + latchReadStarted.countDown() |
| 304 | + latchReadProceed.await() |
| 305 | + super.read() |
| 306 | + } |
| 307 | + } |
232 | 308 | } |
0 commit comments