/*
* 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 org.apache.ignite.internal.processors.hadoop.shuffle;
import org.apache.ignite.internal.processors.hadoop.HadoopJobId;
import org.apache.ignite.internal.processors.hadoop.message.HadoopMessage;
import org.apache.ignite.internal.util.tostring.GridToStringInclude;
import org.apache.ignite.internal.util.typedef.internal.S;
import org.apache.ignite.plugin.extensions.communication.Message;
import org.apache.ignite.plugin.extensions.communication.MessageReader;
import org.apache.ignite.plugin.extensions.communication.MessageWriter;
import java.io.IOException;
import java.io.ObjectInput;
import java.io.ObjectOutput;
import java.nio.ByteBuffer;
/**
* Shuffle finish request.
*/
public class HadoopShuffleFinishRequest implements Message, HadoopMessage {
/** */
private static final long serialVersionUID = 0L;
/** Job ID. */
@GridToStringInclude
private HadoopJobId jobId;
/** Total message count. */
private long msgCnt;
/**
* Default constructor.
*/
public HadoopShuffleFinishRequest() {
// No-op.
}
/**
* Constructor.
*
* @param jobId Job.
* @param msgCnt Message count.
*/
public HadoopShuffleFinishRequest(HadoopJobId jobId, long msgCnt) {
this.jobId = jobId;
this.msgCnt = msgCnt;
}
/**
* @return Job ID.
*/
public HadoopJobId jobId() {
return jobId;
}
/**
* @return Message count.
*/
public long messageCount() {
return msgCnt;
}
/** {@inheritDoc} */
@Override public boolean writeTo(ByteBuffer buf, MessageWriter writer) {
writer.setBuffer(buf);
if (!writer.isHeaderWritten()) {
if (!writer.writeHeader(directType(), fieldsCount()))
return false;
writer.onHeaderWritten();
}
switch (writer.state()) {
case 0:
if (!writer.writeMessage("jobId", jobId))
return false;
writer.incrementState();
case 1:
if (!writer.writeLong("msgCnt", msgCnt))
return false;
writer.incrementState();
}
return true;
}
/** {@inheritDoc} */
@Override public boolean readFrom(ByteBuffer buf, MessageReader reader) {
reader.setBuffer(buf);
if (!reader.beforeMessageRead())
return false;
switch (reader.state()) {
case 0:
jobId = reader.readMessage("jobId");
if (!reader.isLastRead())
return false;
reader.incrementState();
case 1:
msgCnt = reader.readLong("msgCnt");
if (!reader.isLastRead())
return false;
reader.incrementState();
}
return reader.afterMessageRead(HadoopShuffleFinishRequest.class);
}
/** {@inheritDoc} */
@Override public short directType() {
return -40;
}
/** {@inheritDoc} */
@Override public byte fieldsCount() {
return 2;
}
/** {@inheritDoc} */
@Override public void onAckReceived() {
// No-op.
}
/** {@inheritDoc} */
@Override public void writeExternal(ObjectOutput out) throws IOException {
jobId.writeExternal(out);
out.writeLong(msgCnt);
}
/** {@inheritDoc} */
@Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
jobId = new HadoopJobId();
jobId.readExternal(in);
msgCnt = in.readLong();
}
/** {@inheritDoc} */
@Override public String toString() {
return S.toString(HadoopShuffleFinishRequest.class, this);
}
}