You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@ignite.apache.org by GitBox <gi...@apache.org> on 2022/06/03 01:02:50 UTC

[GitHub] [ignite-3] PakhomovAlexander opened a new pull request, #848: IGNITE-16971L: New CLI MVP

PakhomovAlexander opened a new pull request, #848:
URL: https://github.com/apache/ignite-3/pull/848

   https://issues.apache.org/jira/browse/IGNITE-16971


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite-3] kgusakov commented on a diff in pull request #848: IGNITE-16971: New CLI MVP

Posted by GitBox <gi...@apache.org>.
kgusakov commented on code in PR #848:
URL: https://github.com/apache/ignite-3/pull/848#discussion_r890104662


##########
modules/cli/ignite.bat:
##########
@@ -0,0 +1,16 @@
+rem Licensed to the Apache Software Foundation (ASF) under one or more
+rem contributor license agreements.  See the NOTICE file distributed with
+rem this work for additional information regarding copyright ownership.
+rem The ASF licenses this file to You under the Apache License, Version 2.0
+rem (the "License"); you may not use this file except in compliance with
+rem the License.  You may obtain a copy of the License at
+rem
+rem      http://www.apache.org/licenses/LICENSE-2.0
+rem
+rem Unless required by applicable law or agreed to in writing, software
+rem distributed under the License is distributed on an "AS IS" BASIS,
+rem WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+rem See the License for the specific language governing permissions and
+rem limitations under the License.
+
+start java -jar target/ignite.jar

Review Comment:
   Would you add newline at the end of file?



##########
modules/cli/src/integrationTest/java/org/apache/ignite/cli/IntegrationTestBase.java:
##########
@@ -0,0 +1,336 @@
+/*
+ * 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.cli;
+
+import static java.util.stream.Collectors.toList;
+import static org.apache.ignite.internal.testframework.IgniteTestUtils.await;
+import static org.apache.ignite.internal.testframework.IgniteTestUtils.testNodeName;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+
+import io.micronaut.test.extensions.junit5.annotation.MicronautTest;
+import java.io.PrintWriter;
+import java.io.Writer;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.function.Consumer;
+import java.util.stream.IntStream;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgnitionManager;
+import org.apache.ignite.internal.app.IgniteImpl;
+import org.apache.ignite.internal.schema.configuration.SchemaConfigurationConverter;
+import org.apache.ignite.internal.sql.engine.AsyncCursor;
+import org.apache.ignite.internal.sql.engine.AsyncCursor.BatchedResult;
+import org.apache.ignite.internal.testframework.BaseIgniteAbstractTest;
+import org.apache.ignite.internal.testframework.WorkDirectory;
+import org.apache.ignite.internal.testframework.WorkDirectoryExtension;
+import org.apache.ignite.internal.util.IgniteUtils;
+import org.apache.ignite.lang.IgniteLogger;
+import org.apache.ignite.lang.IgniteStringFormatter;
+import org.apache.ignite.schema.SchemaBuilders;
+import org.apache.ignite.schema.definition.ColumnType;
+import org.apache.ignite.schema.definition.TableDefinition;
+import org.apache.ignite.schema.definition.builder.TableDefinitionBuilder;
+import org.apache.ignite.table.RecordView;
+import org.apache.ignite.table.Table;
+import org.apache.ignite.table.Tuple;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.TestInfo;
+import org.junit.jupiter.api.TestInstance;
+import org.junit.jupiter.api.extension.ExtendWith;
+
+/**
+ * Integration test base. Setups ignite cluster per test class and provides useful fixtures and assertions.
+ */
+@ExtendWith(WorkDirectoryExtension.class)
+@TestInstance(TestInstance.Lifecycle.PER_CLASS)
+@MicronautTest
+public class IntegrationTestBase extends BaseIgniteAbstractTest {
+    public static final int DEFAULT_NODES_COUNT = 3;
+    /** Correct ignite cluster url. */
+    protected static final String NODE_URL = "http://localhost:10300";
+    /** Cluster nodes. */
+    protected static final List<Ignite> CLUSTER_NODES = new ArrayList<>();
+    private static final IgniteLogger LOG = IgniteLogger.forClass(IntegrationTestBase.class);
+    /** Base port number. */
+    private static final int BASE_PORT = 3344;
+    /** Nodes bootstrap configuration pattern. */
+    private static final String NODE_BOOTSTRAP_CFG = "{\n"
+            + "  \"network\": {\n"
+            + "    \"port\":{},\n"
+            + "    \"portRange\": 5,\n"
+            + "    \"nodeFinder\":{\n"
+            + "      \"netClusterNodes\": [ {} ]\n"
+            + "    }\n"
+            + "  }\n"
+            + "}";
+    /** Work directory. */
+    @WorkDirectory
+    private static Path WORK_DIR;
+
+    protected static Table createAndPopulateTable() {
+        TableDefinition schTbl1 = SchemaBuilders.tableBuilder("PUBLIC", "PERSON").columns(
+                SchemaBuilders.column("ID", ColumnType.INT32).build(),
+                SchemaBuilders.column("NAME", ColumnType.string()).asNullable(true).build(),
+                SchemaBuilders.column("SALARY", ColumnType.DOUBLE).asNullable(true).build()
+        ).withPrimaryKey("ID").build();
+
+        Table tbl = CLUSTER_NODES.get(0).tables().createTable(schTbl1.canonicalName(), tblCh ->
+                SchemaConfigurationConverter.convert(schTbl1, tblCh)
+                        .changeReplicas(1)
+                        .changePartitions(10)
+        );
+
+        int idx = 0;
+
+        insertData(tbl, new String[]{"ID", "NAME", "SALARY"}, new Object[][]{
+                {idx++, "Igor", 10d},
+                {idx++, null, 15d},
+                {idx++, "Ilya", 15d},
+                {idx++, "Roma", 10d},
+                {idx, "Roma", 10d}
+        });
+
+        return tbl;
+    }
+
+    protected static void createTable(TableDefinitionBuilder tblBld) {
+        TableDefinition schTbl1 = tblBld.build();
+
+        CLUSTER_NODES.get(0).tables().createTable(schTbl1.canonicalName(), tblCh ->
+                SchemaConfigurationConverter.convert(schTbl1, tblCh)
+                        .changeReplicas(1)
+                        .changePartitions(10)
+        );
+    }
+
+    protected static Table table(String canonicalName) {
+        return CLUSTER_NODES.get(0).tables().table(canonicalName);
+    }
+
+    protected static void insertData(String tblName, String[] columnNames, Object[]... tuples) {
+        insertData(CLUSTER_NODES.get(0).tables().table(tblName), columnNames, tuples);
+    }
+
+    protected static void insertData(Table table, String[] columnNames, Object[]... tuples) {
+        RecordView<Tuple> view = table.recordView();
+
+        int batchSize = 128;
+
+        List<Tuple> batch = new ArrayList<>(batchSize);
+        for (Object[] tuple : tuples) {
+            assert tuple != null && tuple.length == columnNames.length;
+
+            Tuple toInsert = Tuple.create();
+
+            for (int i = 0; i < tuple.length; i++) {
+                toInsert.set(columnNames[i], tuple[i]);
+            }
+
+            batch.add(toInsert);
+
+            if (batch.size() == batchSize) {
+                Collection<Tuple> duplicates = view.insertAll(null, batch);
+
+                if (!duplicates.isEmpty()) {
+                    throw new AssertionError("Duplicated rows detected: " + duplicates);
+                }
+
+                batch.clear();
+            }
+        }
+
+        if (!batch.isEmpty()) {
+            view.insertAll(null, batch);
+
+            batch.clear();
+        }
+    }
+
+    protected static void checkData(Table table, String[] columnNames, Object[]... tuples) {
+        RecordView<Tuple> view = table.recordView();
+
+        for (Object[] tuple : tuples) {
+            assert tuple != null && tuple.length == columnNames.length;
+
+            Object id = tuple[0];
+
+            assert id != null : "Primary key cannot be null";
+
+            Tuple row = view.get(null, Tuple.create().set(columnNames[0], id));
+
+            assertNotNull(row);
+
+            for (int i = 0; i < columnNames.length; i++) {
+                assertEquals(tuple[i], row.value(columnNames[i]));
+            }
+        }
+    }
+
+    protected static List<List<Object>> sql(String sql, Object... args) {
+        return getAllFromCursor(
+                ((IgniteImpl) CLUSTER_NODES.get(0)).queryEngine().queryAsync("PUBLIC", sql, args).get(0).join()
+        );
+    }
+
+    private static <T> List<T> reverse(List<T> lst) {
+        List<T> res = new ArrayList<>(lst);
+
+        Collections.reverse(res);
+
+        return res;
+    }
+
+    private static <T> List<T> getAllFromCursor(AsyncCursor<T> cur) {
+        List<T> res = new ArrayList<>();
+        int batchSize = 256;
+
+        var consumer = new Consumer<BatchedResult<T>>() {
+            @Override
+            public void accept(BatchedResult<T> br) {
+                res.addAll(br.items());
+
+                if (br.hasMore()) {
+                    cur.requestNextAsync(batchSize).thenAccept(this);
+                }
+            }
+        };
+
+        await(cur.requestNextAsync(batchSize).thenAccept(consumer));
+        await(cur.closeAsync());
+
+        return res;
+    }
+
+    /**
+     * Before all.
+     *
+     * @param testInfo Test information oject.
+     */
+    @BeforeAll
+    void startNodes(TestInfo testInfo) throws ExecutionException, InterruptedException {
+        String connectNodeAddr = "\"localhost:" + BASE_PORT + '\"';
+
+        List<CompletableFuture<Ignite>> futures = IntStream.range(0, nodes())
+                .mapToObj(i -> {
+                    String nodeName = testNodeName(testInfo, i);
+
+                    String config = IgniteStringFormatter.format(NODE_BOOTSTRAP_CFG, BASE_PORT + i, connectNodeAddr);
+
+                    return IgnitionManager.start(nodeName, config, WORK_DIR.resolve(nodeName));
+                })
+                .collect(toList());
+
+        String metaStorageNodeName = testNodeName(testInfo, 0);
+
+        IgnitionManager.init(metaStorageNodeName, List.of(metaStorageNodeName), "cluster");
+
+        for (CompletableFuture<Ignite> future : futures) {
+            future.get(); //fixme

Review Comment:
   This fixme should be fixed/removed/or linked to issue. Sorry for the bureaucracy, but it's the rule :)



##########
modules/cli/src/main/java/org/apache/ignite/cli/commands/sql/SqlMetaData.java:
##########
@@ -0,0 +1,96 @@
+/*
+ * 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.cli.commands.sql;
+
+/**
+ * Constants for SQL auto-completion.
+ */
+
+public class SqlMetaData {
+
+    /**
+     * SQL reserved keywords.
+     */
+    public static String[] KEYWORDS = {

Review Comment:
   Do we have any plans to share the real actual list of keywords from the ignite sql module? It looks like we will have many custom keywords about data layouts and etc.



##########
modules/cli/src/integrationTest/java/org/apache/ignite/cli/IntegrationTestBase.java:
##########
@@ -0,0 +1,336 @@
+/*
+ * 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.cli;
+
+import static java.util.stream.Collectors.toList;
+import static org.apache.ignite.internal.testframework.IgniteTestUtils.await;
+import static org.apache.ignite.internal.testframework.IgniteTestUtils.testNodeName;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+
+import io.micronaut.test.extensions.junit5.annotation.MicronautTest;
+import java.io.PrintWriter;
+import java.io.Writer;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.function.Consumer;
+import java.util.stream.IntStream;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgnitionManager;
+import org.apache.ignite.internal.app.IgniteImpl;
+import org.apache.ignite.internal.schema.configuration.SchemaConfigurationConverter;
+import org.apache.ignite.internal.sql.engine.AsyncCursor;
+import org.apache.ignite.internal.sql.engine.AsyncCursor.BatchedResult;
+import org.apache.ignite.internal.testframework.BaseIgniteAbstractTest;
+import org.apache.ignite.internal.testframework.WorkDirectory;
+import org.apache.ignite.internal.testframework.WorkDirectoryExtension;
+import org.apache.ignite.internal.util.IgniteUtils;
+import org.apache.ignite.lang.IgniteLogger;
+import org.apache.ignite.lang.IgniteStringFormatter;
+import org.apache.ignite.schema.SchemaBuilders;
+import org.apache.ignite.schema.definition.ColumnType;
+import org.apache.ignite.schema.definition.TableDefinition;
+import org.apache.ignite.schema.definition.builder.TableDefinitionBuilder;
+import org.apache.ignite.table.RecordView;
+import org.apache.ignite.table.Table;
+import org.apache.ignite.table.Tuple;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.TestInfo;
+import org.junit.jupiter.api.TestInstance;
+import org.junit.jupiter.api.extension.ExtendWith;
+
+/**
+ * Integration test base. Setups ignite cluster per test class and provides useful fixtures and assertions.
+ */
+@ExtendWith(WorkDirectoryExtension.class)
+@TestInstance(TestInstance.Lifecycle.PER_CLASS)
+@MicronautTest
+public class IntegrationTestBase extends BaseIgniteAbstractTest {
+    public static final int DEFAULT_NODES_COUNT = 3;
+    /** Correct ignite cluster url. */
+    protected static final String NODE_URL = "http://localhost:10300";

Review Comment:
   According to our guidlines (https://cwiki.apache.org/confluence/display/IGNITE/Coding+Guidelines) all fields' declarations should be separated by empty lines. Pls, check it everywhere in this PR. 



##########
modules/cli/src/integrationTest/java/org/apache/ignite/cli/commands/status/ItStatusReplCommandTest.java:
##########
@@ -0,0 +1,44 @@
+/*
+ * 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.cli.commands.status;
+
+import static org.junit.jupiter.api.Assertions.assertAll;
+
+import org.apache.ignite.cli.commands.CliCommandTestIntegrationBase;
+import org.junit.jupiter.api.Disabled;
+import org.junit.jupiter.api.DisplayName;
+import org.junit.jupiter.api.Test;
+
+/**
+ * Tests for {@link StatusReplCommand}.
+ */
+class ItStatusReplCommandTest extends CliCommandTestIntegrationBase {
+
+    @Test
+    @DisplayName("Should print status when valid cluster url is given")
+    @Disabled // TODO: https://issues.apache.org/jira/browse/IGNITE-17091

Review Comment:
   `@Disabled` has the special form with the reason of disable `@Disabled(https://issues.apache.org/jira/browse/IGNITE-17091)`. It will be better, than sepparate comment, I think. Use it here and in the other disables from PR, pls, if you don't mind.



##########
modules/cli/src/main/java/org/apache/ignite/cli/core/exception/WrappedException.java:
##########
@@ -0,0 +1,33 @@
+/*
+ * 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.cli.core.exception;
+
+/**
+ * Wrapper for checked exception.
+ * This exception will be handled as cause type.
+ */
+public class WrappedException extends RuntimeException {
+    /**
+     * Constructor.
+     *
+     * @param cause cause exception.
+     */
+    public WrappedException(Throwable cause) {
+        super(cause);
+    }
+}

Review Comment:
   No newline - pls check it everywhere



##########
modules/cli/DEVNOTES.md:
##########
@@ -0,0 +1,21 @@
+# Ignite CLI DEVNOTES
+
+## How to build module and add bash/zsh autocompletion to your shell
+
+Cd to module directory and build the module:
+```bash
+cd modules/cli
+mvn clean package

Review Comment:
   On fresh installation, building only cli module isn't enough, ignite-cli-common and etc. packages will not be resolved.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite-3] valepakh commented on a diff in pull request #848: IGNITE-16971: New CLI MVP

Posted by GitBox <gi...@apache.org>.
valepakh commented on code in PR #848:
URL: https://github.com/apache/ignite-3/pull/848#discussion_r891112410


##########
modules/cli/src/integrationTest/java/org/apache/ignite/cli/IntegrationTestBase.java:
##########
@@ -0,0 +1,336 @@
+/*
+ * 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.cli;
+
+import static java.util.stream.Collectors.toList;
+import static org.apache.ignite.internal.testframework.IgniteTestUtils.await;
+import static org.apache.ignite.internal.testframework.IgniteTestUtils.testNodeName;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+
+import io.micronaut.test.extensions.junit5.annotation.MicronautTest;
+import java.io.PrintWriter;
+import java.io.Writer;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.function.Consumer;
+import java.util.stream.IntStream;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgnitionManager;
+import org.apache.ignite.internal.app.IgniteImpl;
+import org.apache.ignite.internal.schema.configuration.SchemaConfigurationConverter;
+import org.apache.ignite.internal.sql.engine.AsyncCursor;
+import org.apache.ignite.internal.sql.engine.AsyncCursor.BatchedResult;
+import org.apache.ignite.internal.testframework.BaseIgniteAbstractTest;
+import org.apache.ignite.internal.testframework.WorkDirectory;
+import org.apache.ignite.internal.testframework.WorkDirectoryExtension;
+import org.apache.ignite.internal.util.IgniteUtils;
+import org.apache.ignite.lang.IgniteLogger;
+import org.apache.ignite.lang.IgniteStringFormatter;
+import org.apache.ignite.schema.SchemaBuilders;
+import org.apache.ignite.schema.definition.ColumnType;
+import org.apache.ignite.schema.definition.TableDefinition;
+import org.apache.ignite.schema.definition.builder.TableDefinitionBuilder;
+import org.apache.ignite.table.RecordView;
+import org.apache.ignite.table.Table;
+import org.apache.ignite.table.Tuple;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.TestInfo;
+import org.junit.jupiter.api.TestInstance;
+import org.junit.jupiter.api.extension.ExtendWith;
+
+/**
+ * Integration test base. Setups ignite cluster per test class and provides useful fixtures and assertions.
+ */
+@ExtendWith(WorkDirectoryExtension.class)
+@TestInstance(TestInstance.Lifecycle.PER_CLASS)
+@MicronautTest
+public class IntegrationTestBase extends BaseIgniteAbstractTest {
+    public static final int DEFAULT_NODES_COUNT = 3;
+    /** Correct ignite cluster url. */
+    protected static final String NODE_URL = "http://localhost:10300";

Review Comment:
   @kgusakov It seems that this document is outdated, according to [IGNITE-15371](https://issues.apache.org/jira/browse/IGNITE-15371) Ignite 3 uses [Google codestyle ](https://google.github.io/styleguide/javaguide.html#s4.6-whitespace) which states that the blank line between two consecutive fields is optional.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite-3] PakhomovAlexander closed pull request #848: IGNITE-16971: New CLI MVP

Posted by GitBox <gi...@apache.org>.
PakhomovAlexander closed pull request #848: IGNITE-16971: New CLI MVP
URL: https://github.com/apache/ignite-3/pull/848


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite-3] valepakh commented on a diff in pull request #848: IGNITE-16971: New CLI MVP

Posted by GitBox <gi...@apache.org>.
valepakh commented on code in PR #848:
URL: https://github.com/apache/ignite-3/pull/848#discussion_r891112410


##########
modules/cli/src/integrationTest/java/org/apache/ignite/cli/IntegrationTestBase.java:
##########
@@ -0,0 +1,336 @@
+/*
+ * 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.cli;
+
+import static java.util.stream.Collectors.toList;
+import static org.apache.ignite.internal.testframework.IgniteTestUtils.await;
+import static org.apache.ignite.internal.testframework.IgniteTestUtils.testNodeName;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+
+import io.micronaut.test.extensions.junit5.annotation.MicronautTest;
+import java.io.PrintWriter;
+import java.io.Writer;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.function.Consumer;
+import java.util.stream.IntStream;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgnitionManager;
+import org.apache.ignite.internal.app.IgniteImpl;
+import org.apache.ignite.internal.schema.configuration.SchemaConfigurationConverter;
+import org.apache.ignite.internal.sql.engine.AsyncCursor;
+import org.apache.ignite.internal.sql.engine.AsyncCursor.BatchedResult;
+import org.apache.ignite.internal.testframework.BaseIgniteAbstractTest;
+import org.apache.ignite.internal.testframework.WorkDirectory;
+import org.apache.ignite.internal.testframework.WorkDirectoryExtension;
+import org.apache.ignite.internal.util.IgniteUtils;
+import org.apache.ignite.lang.IgniteLogger;
+import org.apache.ignite.lang.IgniteStringFormatter;
+import org.apache.ignite.schema.SchemaBuilders;
+import org.apache.ignite.schema.definition.ColumnType;
+import org.apache.ignite.schema.definition.TableDefinition;
+import org.apache.ignite.schema.definition.builder.TableDefinitionBuilder;
+import org.apache.ignite.table.RecordView;
+import org.apache.ignite.table.Table;
+import org.apache.ignite.table.Tuple;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.TestInfo;
+import org.junit.jupiter.api.TestInstance;
+import org.junit.jupiter.api.extension.ExtendWith;
+
+/**
+ * Integration test base. Setups ignite cluster per test class and provides useful fixtures and assertions.
+ */
+@ExtendWith(WorkDirectoryExtension.class)
+@TestInstance(TestInstance.Lifecycle.PER_CLASS)
+@MicronautTest
+public class IntegrationTestBase extends BaseIgniteAbstractTest {
+    public static final int DEFAULT_NODES_COUNT = 3;
+    /** Correct ignite cluster url. */
+    protected static final String NODE_URL = "http://localhost:10300";

Review Comment:
   @kgusakov It seems that this document is outdated, according to [IGNITE-15371](https://issues.apache.org/jira/browse/IGNITE-15371) Ignite 3 uses [Google codestyle ](https://google.github.io/styleguide/javaguide.html#s4.6-whitespace) which states that the blank line between two consecutive fields is optional.
   And here's the link to the [Ignite 3 coding guidelines](https://cwiki.apache.org/confluence/display/IGNITE/Java+Code+Style+Guide)



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org