You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by vo...@apache.org on 2017/03/13 11:51:42 UTC

ignite git commit: Added disco messages.

Repository: ignite
Updated Branches:
  refs/heads/ignite-4565-ddl 7ed196a0e -> dcb8b8511


Added disco messages.


Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/dcb8b851
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/dcb8b851
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/dcb8b851

Branch: refs/heads/ignite-4565-ddl
Commit: dcb8b85119e2571cbd83df2f1e704b1b1e92a3d9
Parents: 7ed196a
Author: devozerov <vo...@gridgain.com>
Authored: Mon Mar 13 14:51:22 2017 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Mon Mar 13 14:51:22 2017 +0300

----------------------------------------------------------------------
 .../processors/query/GridQueryProcessor.java    | 15 ++-
 .../query/ddl/AbstractIndexOperation.java       | 60 ++++++++++++
 .../query/ddl/CreateIndexOperation.java         | 98 ++++++++++++++++++++
 .../query/ddl/DdlAbstractIndexOperation.java    | 60 ------------
 .../query/ddl/DdlCreateIndexOperation.java      | 98 --------------------
 .../query/ddl/DdlDropIndexOperation.java        | 82 ----------------
 .../query/ddl/DropIndexOperation.java           | 82 ++++++++++++++++
 .../ddl/IndexAbstractDiscoveryMessage.java      | 62 +++++++++++++
 .../query/ddl/IndexAckDiscoveryMessage.java     | 54 +++++++++++
 .../query/ddl/IndexInitDiscoveryMessage.java    | 68 ++++++++++++++
 10 files changed, 437 insertions(+), 242 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/dcb8b851/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
index 49e4f38..51b675b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
@@ -56,6 +56,8 @@ import org.apache.ignite.internal.processors.cache.QueryCursorImpl;
 import org.apache.ignite.internal.processors.cache.query.CacheQueryFuture;
 import org.apache.ignite.internal.processors.cache.query.CacheQueryType;
 import org.apache.ignite.internal.processors.cache.query.GridCacheQueryType;
+import org.apache.ignite.internal.processors.query.ddl.CreateIndexOperation;
+import org.apache.ignite.internal.processors.query.ddl.IndexInitDiscoveryMessage;
 import org.apache.ignite.internal.processors.timeout.GridTimeoutProcessor;
 import org.apache.ignite.internal.util.GridSpinBusyLock;
 import org.apache.ignite.internal.util.future.GridFinishedFuture;
@@ -703,12 +705,21 @@ public class GridQueryProcessor extends GridProcessorAdapter {
             UUID opId = UUID.randomUUID();
             QueryIndexClientFuture fut = new QueryIndexClientFuture(opId, idxKey);
 
+            CreateIndexOperation op = new CreateIndexOperation(ctx.localNodeId(), opId, space, tblName, idx,
+                ifNotExists);
+
+            try {
+                ctx.discovery().sendCustomEvent(new IndexInitDiscoveryMessage(op));
+            }
+            catch (IgniteCheckedException e) {
+                return new GridFinishedFuture<>(new IgniteException("Failed to start index create opeartion due to " +
+                    "unexpected exception [space=" + space + ", idxName=" + idxName + ']'));
+            }
+
             QueryIndexClientFuture oldFut = idxCliFuts.put(opId, fut);
 
             assert oldFut == null;
 
-            // TODO: Start discovery.
-
             return fut;
         }
         finally {

http://git-wip-us.apache.org/repos/asf/ignite/blob/dcb8b851/modules/core/src/main/java/org/apache/ignite/internal/processors/query/ddl/AbstractIndexOperation.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/ddl/AbstractIndexOperation.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/ddl/AbstractIndexOperation.java
new file mode 100644
index 0000000..cfa8f24
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/ddl/AbstractIndexOperation.java
@@ -0,0 +1,60 @@
+/*
+ * 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.query.ddl;
+
+import java.io.Serializable;
+import java.util.UUID;
+
+/**
+ * Abstract operation on index.
+ */
+public abstract class AbstractIndexOperation implements Serializable {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** ID of node that initiated this operation. */
+    private final UUID cliNodeId;
+
+    /** Operation ID. */
+    private final UUID opId;
+
+    /**
+     * Constructor.
+     *
+     * @param cliNodeId Client node ID.
+     * @param opId Operation ID.
+     */
+    public AbstractIndexOperation(UUID cliNodeId, UUID opId) {
+        this.cliNodeId = cliNodeId;
+        this.opId = opId;
+    }
+
+    /**
+     * @return Client node ID.
+     */
+    public UUID clientNodeId() {
+        return cliNodeId;
+    }
+
+    /**
+     * @return Operation id.
+     */
+    public UUID operationId() {
+        return opId;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/dcb8b851/modules/core/src/main/java/org/apache/ignite/internal/processors/query/ddl/CreateIndexOperation.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/ddl/CreateIndexOperation.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/ddl/CreateIndexOperation.java
new file mode 100644
index 0000000..125d0e2
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/ddl/CreateIndexOperation.java
@@ -0,0 +1,98 @@
+/*
+ * 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.query.ddl;
+
+import org.apache.ignite.cache.QueryIndex;
+import org.apache.ignite.internal.util.tostring.GridToStringInclude;
+import org.apache.ignite.internal.util.typedef.internal.S;
+
+import java.util.UUID;
+
+/**
+ * Arguments for {@code CREATE INDEX}.
+ */
+public class CreateIndexOperation extends AbstractIndexOperation {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** Space. */
+    private final String space;
+
+    /** Table name. */
+    private final String tblName;
+
+    /** Index. */
+    @GridToStringInclude
+    private final QueryIndex idx;
+
+    /** Ignore operation if index exists. */
+    private final boolean ifNotExists;
+
+    /**
+     * Constructor.
+     *
+     * @param cliNodeId Id of node that initiated this operation.
+     * @param opId Operation id.
+     * @param space Space.
+     * @param tblName Table name.
+     * @param idx Index params.
+     * @param ifNotExists Ignore operation if index exists.
+     */
+    public CreateIndexOperation(UUID cliNodeId, UUID opId, String space, String tblName, QueryIndex idx,
+        boolean ifNotExists) {
+        super(cliNodeId, opId);
+
+        this.space = space;
+        this.tblName = tblName;
+        this.idx = idx;
+        this.ifNotExists = ifNotExists;
+    }
+
+    /**
+     * @return Index params.
+     */
+    public QueryIndex index() {
+        return idx;
+    }
+
+    /**
+     * @return Schema name.
+     */
+    public String space() {
+        return space;
+    }
+
+    /**
+     * @return Table name.
+     */
+    public String tableName() {
+        return tblName;
+    }
+
+    /**
+     * @return Ignore operation if index exists.
+     */
+    public boolean ifNotExists() {
+        return ifNotExists;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(CreateIndexOperation.class, this);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/dcb8b851/modules/core/src/main/java/org/apache/ignite/internal/processors/query/ddl/DdlAbstractIndexOperation.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/ddl/DdlAbstractIndexOperation.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/ddl/DdlAbstractIndexOperation.java
deleted file mode 100644
index c8fe544..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/ddl/DdlAbstractIndexOperation.java
+++ /dev/null
@@ -1,60 +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 org.apache.ignite.internal.processors.query.ddl;
-
-import java.io.Serializable;
-import java.util.UUID;
-
-/**
- * DDL index operation.
- */
-public abstract class DdlAbstractIndexOperation implements Serializable {
-    /** */
-    private static final long serialVersionUID = 0L;
-
-    /** ID of node that initiated this operation. */
-    private final UUID cliNodeId;
-
-    /** Operation ID. */
-    private final UUID opId;
-
-    /**
-     * Constructor.
-     *
-     * @param cliNodeId Client node ID.
-     * @param opId Operation ID.
-     */
-    public DdlAbstractIndexOperation(UUID cliNodeId, UUID opId) {
-        this.cliNodeId = cliNodeId;
-        this.opId = opId;
-    }
-
-    /**
-     * @return Client node ID.
-     */
-    public UUID clientNodeId() {
-        return cliNodeId;
-    }
-
-    /**
-     * @return Operation id.
-     */
-    public UUID operationId() {
-        return opId;
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/dcb8b851/modules/core/src/main/java/org/apache/ignite/internal/processors/query/ddl/DdlCreateIndexOperation.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/ddl/DdlCreateIndexOperation.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/ddl/DdlCreateIndexOperation.java
deleted file mode 100644
index dc8e85a..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/ddl/DdlCreateIndexOperation.java
+++ /dev/null
@@ -1,98 +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 org.apache.ignite.internal.processors.query.ddl;
-
-import org.apache.ignite.cache.QueryIndex;
-import org.apache.ignite.internal.util.tostring.GridToStringInclude;
-import org.apache.ignite.internal.util.typedef.internal.S;
-
-import java.util.UUID;
-
-/**
- * Arguments for {@code CREATE INDEX}.
- */
-public class DdlCreateIndexOperation extends DdlAbstractIndexOperation {
-    /** */
-    private static final long serialVersionUID = 0L;
-
-    /** Space. */
-    private final String space;
-
-    /** Table name. */
-    private final String tblName;
-
-    /** Index. */
-    @GridToStringInclude
-    private final QueryIndex idx;
-
-    /** Ignore operation if index exists. */
-    private final boolean ifNotExists;
-
-    /**
-     * Constructor.
-     *
-     * @param cliNodeId Id of node that initiated this operation.
-     * @param opId Operation id.
-     * @param space Space.
-     * @param tblName Table name.
-     * @param idx Index params.
-     * @param ifNotExists Ignore operation if index exists.
-     */
-    DdlCreateIndexOperation(UUID cliNodeId, UUID opId, String space, String tblName, QueryIndex idx,
-        boolean ifNotExists) {
-        super(cliNodeId, opId);
-
-        this.space = space;
-        this.tblName = tblName;
-        this.idx = idx;
-        this.ifNotExists = ifNotExists;
-    }
-
-    /**
-     * @return Index params.
-     */
-    public QueryIndex index() {
-        return idx;
-    }
-
-    /**
-     * @return Schema name.
-     */
-    public String space() {
-        return space;
-    }
-
-    /**
-     * @return Table name.
-     */
-    public String tableName() {
-        return tblName;
-    }
-
-    /**
-     * @return Ignore operation if index exists.
-     */
-    public boolean ifNotExists() {
-        return ifNotExists;
-    }
-
-    /** {@inheritDoc} */
-    @Override public String toString() {
-        return S.toString(DdlCreateIndexOperation.class, this);
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/dcb8b851/modules/core/src/main/java/org/apache/ignite/internal/processors/query/ddl/DdlDropIndexOperation.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/ddl/DdlDropIndexOperation.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/ddl/DdlDropIndexOperation.java
deleted file mode 100644
index 02467f3..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/ddl/DdlDropIndexOperation.java
+++ /dev/null
@@ -1,82 +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 org.apache.ignite.internal.processors.query.ddl;
-
-import org.apache.ignite.internal.util.typedef.internal.S;
-
-import java.util.UUID;
-
-/**
- * Arguments for {@code CREATE INDEX}.
- */
-public class DdlDropIndexOperation extends DdlAbstractIndexOperation {
-    /** */
-    private static final long serialVersionUID = 0L;
-
-    /** Space. */
-    private final String space;
-
-    /** Index name. */
-    private final String idxName;
-
-    /** Ignore operation if index doesn't exist. */
-    private final boolean ifExists;
-
-    /**
-     * Constructor.
-     *
-     * @param cliNodeId Client node ID.
-     * @param opId Operation id.
-     * @param space Space.
-     * @param idxName Index name.
-     * @param ifExists Ignore operation if index doesn't exist.
-     */
-    DdlDropIndexOperation(UUID cliNodeId, UUID opId, String space, String idxName, boolean ifExists) {
-        super(cliNodeId, opId);
-
-        this.space = space;
-        this.idxName = idxName;
-        this.ifExists = ifExists;
-    }
-
-    /**
-     * @return Space.
-     */
-    public String space() {
-        return space;
-    }
-
-    /**
-     * @return Index name.
-     */
-    public String indexName() {
-        return idxName;
-    }
-
-    /**
-     * @return Ignore operation if index doesn't exist.
-     */
-    public boolean ifExists() {
-        return ifExists;
-    }
-
-    /** {@inheritDoc} */
-    @Override public String toString() {
-        return S.toString(DdlDropIndexOperation.class, this);
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/dcb8b851/modules/core/src/main/java/org/apache/ignite/internal/processors/query/ddl/DropIndexOperation.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/ddl/DropIndexOperation.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/ddl/DropIndexOperation.java
new file mode 100644
index 0000000..14c3ffd
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/ddl/DropIndexOperation.java
@@ -0,0 +1,82 @@
+/*
+ * 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.query.ddl;
+
+import org.apache.ignite.internal.util.typedef.internal.S;
+
+import java.util.UUID;
+
+/**
+ * Arguments for {@code CREATE INDEX}.
+ */
+public class DropIndexOperation extends AbstractIndexOperation {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** Space. */
+    private final String space;
+
+    /** Index name. */
+    private final String idxName;
+
+    /** Ignore operation if index doesn't exist. */
+    private final boolean ifExists;
+
+    /**
+     * Constructor.
+     *
+     * @param cliNodeId Client node ID.
+     * @param opId Operation id.
+     * @param space Space.
+     * @param idxName Index name.
+     * @param ifExists Ignore operation if index doesn't exist.
+     */
+    DropIndexOperation(UUID cliNodeId, UUID opId, String space, String idxName, boolean ifExists) {
+        super(cliNodeId, opId);
+
+        this.space = space;
+        this.idxName = idxName;
+        this.ifExists = ifExists;
+    }
+
+    /**
+     * @return Space.
+     */
+    public String space() {
+        return space;
+    }
+
+    /**
+     * @return Index name.
+     */
+    public String indexName() {
+        return idxName;
+    }
+
+    /**
+     * @return Ignore operation if index doesn't exist.
+     */
+    public boolean ifExists() {
+        return ifExists;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(DropIndexOperation.class, this);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/dcb8b851/modules/core/src/main/java/org/apache/ignite/internal/processors/query/ddl/IndexAbstractDiscoveryMessage.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/ddl/IndexAbstractDiscoveryMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/ddl/IndexAbstractDiscoveryMessage.java
new file mode 100644
index 0000000..b7599e6
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/ddl/IndexAbstractDiscoveryMessage.java
@@ -0,0 +1,62 @@
+/*
+ * 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.query.ddl;
+
+import org.apache.ignite.internal.managers.discovery.DiscoveryCustomMessage;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.lang.IgniteUuid;
+
+/**
+ * Abstract discovery message for index operations.
+ */
+public abstract class IndexAbstractDiscoveryMessage implements DiscoveryCustomMessage {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** ID */
+    private final IgniteUuid id = IgniteUuid.randomUuid();
+
+    /** Operation. */
+    private final AbstractIndexOperation op;
+
+    /**
+     * Constructor.
+     *
+     * @param op Operation.
+     */
+    protected IndexAbstractDiscoveryMessage(AbstractIndexOperation op) {
+        this.op = op;
+    }
+
+    /** {@inheritDoc} */
+    @Override public IgniteUuid id() {
+        return id;
+    }
+
+    /**
+     * @return Operation.
+     */
+    public AbstractIndexOperation operation() {
+        return op;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(IndexAbstractDiscoveryMessage.class, this);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/dcb8b851/modules/core/src/main/java/org/apache/ignite/internal/processors/query/ddl/IndexAckDiscoveryMessage.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/ddl/IndexAckDiscoveryMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/ddl/IndexAckDiscoveryMessage.java
new file mode 100644
index 0000000..44a89fc
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/ddl/IndexAckDiscoveryMessage.java
@@ -0,0 +1,54 @@
+/*
+ * 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.query.ddl;
+
+import org.apache.ignite.internal.managers.discovery.DiscoveryCustomMessage;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * {@code ACK} message which triggers local index create/drop.
+ */
+public class IndexAckDiscoveryMessage extends IndexAbstractDiscoveryMessage {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /**
+     * Constructor.
+     *
+     * @param op Original operation.
+     */
+    public IndexAckDiscoveryMessage(AbstractIndexOperation op) {
+        super(op);
+    }
+
+    /** {@inheritDoc} */
+    @Nullable @Override public DiscoveryCustomMessage ackMessage() {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean isMutable() {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(IndexAckDiscoveryMessage.class, this);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/dcb8b851/modules/core/src/main/java/org/apache/ignite/internal/processors/query/ddl/IndexInitDiscoveryMessage.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/ddl/IndexInitDiscoveryMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/ddl/IndexInitDiscoveryMessage.java
new file mode 100644
index 0000000..b6d3a2f
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/ddl/IndexInitDiscoveryMessage.java
@@ -0,0 +1,68 @@
+/*
+ * 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.query.ddl;
+
+import org.apache.ignite.internal.ContextAware;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.managers.discovery.DiscoveryCustomMessage;
+import org.apache.ignite.internal.util.tostring.GridToStringExclude;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * {@code INIT} part of a distributed index create/drop operation.
+ */
+public class IndexInitDiscoveryMessage extends IndexAbstractDiscoveryMessage implements ContextAware {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** Kernal context. */
+    @GridToStringExclude
+    private transient GridKernalContext ctx;
+
+    /**
+     * Constructor.
+     *
+     * @param op Operation.
+     */
+    public IndexInitDiscoveryMessage(AbstractIndexOperation op) {
+        super(op);
+    }
+
+    /** {@inheritDoc} */
+    @Nullable @Override public DiscoveryCustomMessage ackMessage() {
+        // TODO: Ask indexing for ack message
+
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean isMutable() {
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void context(GridKernalContext ctx) {
+        this.ctx = ctx;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(IndexInitDiscoveryMessage.class, this);
+    }
+}