Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

HCD-481 : Adding reconnection logic added for HDFS2 sink connector #531

Open
wants to merge 2 commits into
base: 10.0.x
Choose a base branch
from
Open
Show file tree
Hide file tree
Changes from 1 commit
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,8 @@
import io.confluent.connect.hdfs.storage.HdfsStorage;
import io.confluent.connect.storage.format.RecordWriter;
import java.io.OutputStream;

import org.apache.avro.AvroRuntimeException;
import org.apache.avro.file.CodecFactory;
import org.apache.avro.file.DataFileWriter;
import org.apache.avro.generic.GenericDatumWriter;
Expand Down Expand Up @@ -84,6 +86,14 @@ public void write(SinkRecord record) {
}
} catch (IOException e) {
throw new DataException(e);
} catch (AvroRuntimeException ex) {
// Making schema null to invalidate file corrupted due to connection failure
schema = null;
/*
Wrapped into ConnectException so that setRetryTimeout() is called in the
TopicPartitionWriter class to retry the records.
*/
throw new ConnectException(ex);
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,12 +19,15 @@
import io.confluent.connect.storage.format.RecordWriter;
import io.confluent.connect.storage.format.RecordWriterProvider;
import io.confluent.connect.storage.hive.HiveSchemaConverter;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hdfs.protocol.AlreadyBeingCreatedException;
import org.apache.hadoop.hive.ql.io.orc.OrcFile;
import org.apache.hadoop.hive.ql.io.orc.OrcStruct;
import org.apache.hadoop.hive.ql.io.orc.Writer;
import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
import org.apache.hadoop.ipc.RemoteException;
import org.apache.kafka.connect.data.Schema;
import org.apache.kafka.connect.data.Struct;
import org.apache.kafka.connect.errors.ConnectException;
Expand All @@ -33,6 +36,7 @@
import org.slf4j.LoggerFactory;

import java.io.IOException;
import java.net.URI;

public class OrcRecordWriterProvider implements RecordWriterProvider<HdfsSinkConnectorConfig> {

Expand Down Expand Up @@ -99,6 +103,24 @@ public void preFooterWrite(OrcFile.WriterContext writerContext) {
);
}
} catch (IOException e) {
if (e instanceof RemoteException || e instanceof AlreadyBeingCreatedException) {
try {
/*
The current version of OrcFile.WriterOptions does not supports file overwriting
therefore deleting tmp file whenever`AlreadyBeingCreatedException` is thrown.
*/
FileSystem fileSystem = FileSystem.get(
URI.create(conf.getUrl()),
conf.getHadoopConfiguration()
);
fileSystem.delete(path, true);
Copy link

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Not too familiar with HDFS. Why are we deleting the file for ORC here? Why is schema=null not enough like with AVRO

Copy link
Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The current version of org.apache.hadoop.hive.ql.io.orc.OrcFile.WriterOptions does not supports file overwriting therefore deleting tmp file wheneverAlreadyBeingCreatedException is thrown.

} catch (IOException ex) {
Copy link

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Can we log this error message as well?

Copy link
Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Have added the exception to the log below.

// connection refused to the HDFS cluster.
log.debug("Connection failed while deleting the tmp file from the HDFS cluster.");
}
}
// Making schema null to invalidate corrupted file due to connection failure
schema = null;
throw new ConnectException("Failed to write record: ", e);
}
}
Expand Down